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

2024-05-23 Thread Jane Chan
Hi Leonard,

Thank you for the feedback and the improvement.

If there are no further comments or concerns, I would like to initiate a
vote on this.

Best,
Jane

On Wed, May 22, 2024 at 9:24 PM Leonard Xu  wrote:

> Thanks Jane for the refine work, +1 from my side.
> I adjusted the table format of FLIP so that it can display all content in
> one page.
>
> Best,
> Leonard
>
>
> > 2024年5月22日 下午3:42,Jane Chan  写道:
> >
> > Hi Lincoln,
> >
> > Thanks for your suggestion. I've reviewed the comments from the previous
> PR
> > review[1], and the agreement at the time was that any configuration
> options
> > not included in ExecutionConfigOptions and OptimizerConfigOptions should
> > have the Experimental annotation explicitly added. Since this annotation
> > has been relatively stable from 1.9.0 until now, you make a valid point,
> > and we can elevate it to the PublicEvolving level.
> >
> > Please let me know if you have any questions.
> >
> > [1] https://github.com/apache/flink/pull/8980
> >
> > Best,
> > Jane
> >
> > On Tue, May 21, 2024 at 10:25 PM Lincoln Lee 
> wrote:
> >
> >> Hi Jane,
> >>
> >> Thanks for the updates!
> >>
> >> Just one small comment on the options in IncrementalAggregateRule
> >> & RelNodeBlock, should we also change the API level from Experimental
> >> to PublicEvolving?
> >>
> >>
> >> Best,
> >> Lincoln Lee
> >>
> >>
> >> Jane Chan  于2024年5月21日周二 16:41写道:
> >>
> >>> Hi all,
> >>>
> >>> Thanks for your valuable feedback!
> >>>
> >>> To @Xuannan
> >>>
> >>> For options to be moved to another module/package, I think we have to
>  mark the old option deprecated in 1.20 for it to be removed in 2.0,
>  according to the API compatibility guarantees[1]. We can introduce the
>  new option in 1.20 with the same option key in the intended class.
> >>>
> >>>
> >>> Good point, fixed.
> >>>
> >>> To @Lincoln and @Benchao
> >>>
> >>> Thanks for sharing the insights into the historical context of which I
> >> was
> >>> unaware. I've reorganized the sheet.
> >>>
> >>> 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on
> TVF
>  window, so it's recommended to keep it untouched for now and follow up
> >> in
>  FLINK-29692
> >>>
> >>>
> >>> How to tackle the configuration is up to whether to remove the legacy
> >>> window aggregate in 2.0, and I've updated the FLIP to leverage this
> part
> >> to
> >>> FLINK-29692.
> >>>
> >>> Please let me know if that answers your questions or if you have other
> >>> comments.
> >>>
> >>> Best,
> >>> Jane
> >>>
> >>>
> >>> On Mon, May 20, 2024 at 1:52 PM Ron Liu  wrote:
> >>>
>  Hi, Lincoln
> 
> > 2. Regarding the options in HashAggCodeGenerator, since this new
> >>> feature
>  has gone
>  through a couple of release cycles and could be considered for
>  PublicEvolving now,
>  cc @Ron Liu   WDYT?
> 
>  Thanks for cc'ing me,  +1 for public these options now.
> 
>  Best,
>  Ron
> 
>  Benchao Li  于2024年5月20日周一 13:08写道:
> 
> > I agree with Lincoln about the experimental features.
> >
> > Some of these configurations do not even have proper implementation,
> > take 'table.exec.range-sort.enabled' as an example, there was a
> > discussion[1] about it before.
> >
> > [1] https://lists.apache.org/thread/q5h3obx36pf9po28r0jzmwnmvtyjmwdr
> >
> > Lincoln Lee  于2024年5月20日周一 12:01写道:
> >>
> >> Hi Jane,
> >>
> >> Thanks for the proposal!
> >>
> >> +1 for the changes except for these annotated as experimental ones.
> >>
> >> For the options annotated as experimental,
> >>
> >> +1 for the moving of IncrementalAggregateRule & RelNodeBlock.
> >>
> >> For the rest of the options, there are some suggestions:
> >>
> >> 1. for the batch related parameters, it's recommended to either
> >>> delete
> >> them (leaving the necessary defaults value in place) or leave them
> >> as
> > they
> >> are. Including:
> >> FlinkRelMdRowCount
> >> FlinkRexUtil
> >> BatchPhysicalSortRule
> >> JoinDeriveNullFilterRule
> >> BatchPhysicalJoinRuleBase
> >> BatchPhysicalSortMergeJoinRule
> >>
> >> What I understand about the history of these options is that they
> >>> were
> > once
> >> used for fine
> >> tuning for tpc testing, and the current flink planner no longer
> >>> relies
>  on
> >> these internal
> >> options when testing tpc[1]. In addition, these options are too
> >>> obscure
> > for
> >> SQL users,
> >> and some of them are actually magic numbers.
> >>
> >> 2. Regarding the options in HashAggCodeGenerator, since this new
>  feature
> >> has gone
> >> through a couple of release cycles and could be considered for
> >> PublicEvolving now,
> >> cc @Ron Liu   WDYT?
> >>
> >> 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported
> >> on
>  TVF
> >> window, so
> >> it's recommended to 

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

2024-05-22 Thread Leonard Xu
Thanks Jane for the refine work, +1 from my side. 
I adjusted the table format of FLIP so that it can display all content in one 
page.

Best,
Leonard


> 2024年5月22日 下午3:42,Jane Chan  写道:
> 
> Hi Lincoln,
> 
> Thanks for your suggestion. I've reviewed the comments from the previous PR
> review[1], and the agreement at the time was that any configuration options
> not included in ExecutionConfigOptions and OptimizerConfigOptions should
> have the Experimental annotation explicitly added. Since this annotation
> has been relatively stable from 1.9.0 until now, you make a valid point,
> and we can elevate it to the PublicEvolving level.
> 
> Please let me know if you have any questions.
> 
> [1] https://github.com/apache/flink/pull/8980
> 
> Best,
> Jane
> 
> On Tue, May 21, 2024 at 10:25 PM Lincoln Lee  wrote:
> 
>> Hi Jane,
>> 
>> Thanks for the updates!
>> 
>> Just one small comment on the options in IncrementalAggregateRule
>> & RelNodeBlock, should we also change the API level from Experimental
>> to PublicEvolving?
>> 
>> 
>> Best,
>> Lincoln Lee
>> 
>> 
>> Jane Chan  于2024年5月21日周二 16:41写道:
>> 
>>> Hi all,
>>> 
>>> Thanks for your valuable feedback!
>>> 
>>> To @Xuannan
>>> 
>>> For options to be moved to another module/package, I think we have to
 mark the old option deprecated in 1.20 for it to be removed in 2.0,
 according to the API compatibility guarantees[1]. We can introduce the
 new option in 1.20 with the same option key in the intended class.
>>> 
>>> 
>>> Good point, fixed.
>>> 
>>> To @Lincoln and @Benchao
>>> 
>>> Thanks for sharing the insights into the historical context of which I
>> was
>>> unaware. I've reorganized the sheet.
>>> 
>>> 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on TVF
 window, so it's recommended to keep it untouched for now and follow up
>> in
 FLINK-29692
>>> 
>>> 
>>> How to tackle the configuration is up to whether to remove the legacy
>>> window aggregate in 2.0, and I've updated the FLIP to leverage this part
>> to
>>> FLINK-29692.
>>> 
>>> Please let me know if that answers your questions or if you have other
>>> comments.
>>> 
>>> Best,
>>> Jane
>>> 
>>> 
>>> On Mon, May 20, 2024 at 1:52 PM Ron Liu  wrote:
>>> 
 Hi, Lincoln
 
> 2. Regarding the options in HashAggCodeGenerator, since this new
>>> feature
 has gone
 through a couple of release cycles and could be considered for
 PublicEvolving now,
 cc @Ron Liu   WDYT?
 
 Thanks for cc'ing me,  +1 for public these options now.
 
 Best,
 Ron
 
 Benchao Li  于2024年5月20日周一 13:08写道:
 
> I agree with Lincoln about the experimental features.
> 
> Some of these configurations do not even have proper implementation,
> take 'table.exec.range-sort.enabled' as an example, there was a
> discussion[1] about it before.
> 
> [1] https://lists.apache.org/thread/q5h3obx36pf9po28r0jzmwnmvtyjmwdr
> 
> Lincoln Lee  于2024年5月20日周一 12:01写道:
>> 
>> Hi Jane,
>> 
>> Thanks for the proposal!
>> 
>> +1 for the changes except for these annotated as experimental ones.
>> 
>> For the options annotated as experimental,
>> 
>> +1 for the moving of IncrementalAggregateRule & RelNodeBlock.
>> 
>> For the rest of the options, there are some suggestions:
>> 
>> 1. for the batch related parameters, it's recommended to either
>>> delete
>> them (leaving the necessary defaults value in place) or leave them
>> as
> they
>> are. Including:
>> FlinkRelMdRowCount
>> FlinkRexUtil
>> BatchPhysicalSortRule
>> JoinDeriveNullFilterRule
>> BatchPhysicalJoinRuleBase
>> BatchPhysicalSortMergeJoinRule
>> 
>> What I understand about the history of these options is that they
>>> were
> once
>> used for fine
>> tuning for tpc testing, and the current flink planner no longer
>>> relies
 on
>> these internal
>> options when testing tpc[1]. In addition, these options are too
>>> obscure
> for
>> SQL users,
>> and some of them are actually magic numbers.
>> 
>> 2. Regarding the options in HashAggCodeGenerator, since this new
 feature
>> has gone
>> through a couple of release cycles and could be considered for
>> PublicEvolving now,
>> cc @Ron Liu   WDYT?
>> 
>> 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported
>> on
 TVF
>> window, so
>> it's recommended to keep it untouched for now and follow up in
>> FLINK-29692[2]. cc @Xuyang 
>> 
>> [1]
>> 
> 
 
>>> 
>> https://github.com/ververica/flink-sql-benchmark/blob/master/tools/common/flink-conf.yaml
>> [2] https://issues.apache.org/jira/browse/FLINK-29692
>> 
>> 
>> Best,
>> Lincoln Lee
>> 
>> 
>> Yubin Li  于2024年5月17日周五 10:49写道:
>> 
>>> Hi Jane,
>>> 
>>> Thank Jane for driving this proposal !
>>> 
>>> This makes se

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

2024-05-22 Thread Jane Chan
Hi Lincoln,

Thanks for your suggestion. I've reviewed the comments from the previous PR
review[1], and the agreement at the time was that any configuration options
not included in ExecutionConfigOptions and OptimizerConfigOptions should
have the Experimental annotation explicitly added. Since this annotation
has been relatively stable from 1.9.0 until now, you make a valid point,
and we can elevate it to the PublicEvolving level.

Please let me know if you have any questions.

[1] https://github.com/apache/flink/pull/8980

Best,
Jane

On Tue, May 21, 2024 at 10:25 PM Lincoln Lee  wrote:

> Hi Jane,
>
> Thanks for the updates!
>
> Just one small comment on the options in IncrementalAggregateRule
> & RelNodeBlock, should we also change the API level from Experimental
> to PublicEvolving?
>
>
> Best,
> Lincoln Lee
>
>
> Jane Chan  于2024年5月21日周二 16:41写道:
>
> > Hi all,
> >
> > Thanks for your valuable feedback!
> >
> > To @Xuannan
> >
> > For options to be moved to another module/package, I think we have to
> > > mark the old option deprecated in 1.20 for it to be removed in 2.0,
> > > according to the API compatibility guarantees[1]. We can introduce the
> > > new option in 1.20 with the same option key in the intended class.
> >
> >
> > Good point, fixed.
> >
> > To @Lincoln and @Benchao
> >
> > Thanks for sharing the insights into the historical context of which I
> was
> > unaware. I've reorganized the sheet.
> >
> > 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on TVF
> > > window, so it's recommended to keep it untouched for now and follow up
> in
> > > FLINK-29692
> >
> >
> > How to tackle the configuration is up to whether to remove the legacy
> > window aggregate in 2.0, and I've updated the FLIP to leverage this part
> to
> > FLINK-29692.
> >
> > Please let me know if that answers your questions or if you have other
> > comments.
> >
> > Best,
> > Jane
> >
> >
> > On Mon, May 20, 2024 at 1:52 PM Ron Liu  wrote:
> >
> > > Hi, Lincoln
> > >
> > > >  2. Regarding the options in HashAggCodeGenerator, since this new
> > feature
> > > has gone
> > > through a couple of release cycles and could be considered for
> > > PublicEvolving now,
> > > cc @Ron Liu   WDYT?
> > >
> > > Thanks for cc'ing me,  +1 for public these options now.
> > >
> > > Best,
> > > Ron
> > >
> > > Benchao Li  于2024年5月20日周一 13:08写道:
> > >
> > > > I agree with Lincoln about the experimental features.
> > > >
> > > > Some of these configurations do not even have proper implementation,
> > > > take 'table.exec.range-sort.enabled' as an example, there was a
> > > > discussion[1] about it before.
> > > >
> > > > [1] https://lists.apache.org/thread/q5h3obx36pf9po28r0jzmwnmvtyjmwdr
> > > >
> > > > Lincoln Lee  于2024年5月20日周一 12:01写道:
> > > > >
> > > > > Hi Jane,
> > > > >
> > > > > Thanks for the proposal!
> > > > >
> > > > > +1 for the changes except for these annotated as experimental ones.
> > > > >
> > > > > For the options annotated as experimental,
> > > > >
> > > > > +1 for the moving of IncrementalAggregateRule & RelNodeBlock.
> > > > >
> > > > > For the rest of the options, there are some suggestions:
> > > > >
> > > > > 1. for the batch related parameters, it's recommended to either
> > delete
> > > > > them (leaving the necessary defaults value in place) or leave them
> as
> > > > they
> > > > > are. Including:
> > > > > FlinkRelMdRowCount
> > > > > FlinkRexUtil
> > > > > BatchPhysicalSortRule
> > > > > JoinDeriveNullFilterRule
> > > > > BatchPhysicalJoinRuleBase
> > > > > BatchPhysicalSortMergeJoinRule
> > > > >
> > > > > What I understand about the history of these options is that they
> > were
> > > > once
> > > > > used for fine
> > > > > tuning for tpc testing, and the current flink planner no longer
> > relies
> > > on
> > > > > these internal
> > > > > options when testing tpc[1]. In addition, these options are too
> > obscure
> > > > for
> > > > > SQL users,
> > > > > and some of them are actually magic numbers.
> > > > >
> > > > > 2. Regarding the options in HashAggCodeGenerator, since this new
> > > feature
> > > > > has gone
> > > > > through a couple of release cycles and could be considered for
> > > > > PublicEvolving now,
> > > > > cc @Ron Liu   WDYT?
> > > > >
> > > > > 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported
> on
> > > TVF
> > > > > window, so
> > > > > it's recommended to keep it untouched for now and follow up in
> > > > > FLINK-29692[2]. cc @Xuyang 
> > > > >
> > > > > [1]
> > > > >
> > > >
> > >
> >
> https://github.com/ververica/flink-sql-benchmark/blob/master/tools/common/flink-conf.yaml
> > > > > [2] https://issues.apache.org/jira/browse/FLINK-29692
> > > > >
> > > > >
> > > > > Best,
> > > > > Lincoln Lee
> > > > >
> > > > >
> > > > > Yubin Li  于2024年5月17日周五 10:49写道:
> > > > >
> > > > > > Hi Jane,
> > > > > >
> > > > > > Thank Jane for driving this proposal !
> > > > > >
> > > > > > This makes sense for users, +1 for that.
> > > > > >
> > > > > > Best,
> > > > > >

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

2024-05-21 Thread Lincoln Lee
Hi Jane,

Thanks for the updates!

Just one small comment on the options in IncrementalAggregateRule
& RelNodeBlock, should we also change the API level from Experimental
to PublicEvolving?


Best,
Lincoln Lee


Jane Chan  于2024年5月21日周二 16:41写道:

> Hi all,
>
> Thanks for your valuable feedback!
>
> To @Xuannan
>
> For options to be moved to another module/package, I think we have to
> > mark the old option deprecated in 1.20 for it to be removed in 2.0,
> > according to the API compatibility guarantees[1]. We can introduce the
> > new option in 1.20 with the same option key in the intended class.
>
>
> Good point, fixed.
>
> To @Lincoln and @Benchao
>
> Thanks for sharing the insights into the historical context of which I was
> unaware. I've reorganized the sheet.
>
> 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on TVF
> > window, so it's recommended to keep it untouched for now and follow up in
> > FLINK-29692
>
>
> How to tackle the configuration is up to whether to remove the legacy
> window aggregate in 2.0, and I've updated the FLIP to leverage this part to
> FLINK-29692.
>
> Please let me know if that answers your questions or if you have other
> comments.
>
> Best,
> Jane
>
>
> On Mon, May 20, 2024 at 1:52 PM Ron Liu  wrote:
>
> > Hi, Lincoln
> >
> > >  2. Regarding the options in HashAggCodeGenerator, since this new
> feature
> > has gone
> > through a couple of release cycles and could be considered for
> > PublicEvolving now,
> > cc @Ron Liu   WDYT?
> >
> > Thanks for cc'ing me,  +1 for public these options now.
> >
> > Best,
> > Ron
> >
> > Benchao Li  于2024年5月20日周一 13:08写道:
> >
> > > I agree with Lincoln about the experimental features.
> > >
> > > Some of these configurations do not even have proper implementation,
> > > take 'table.exec.range-sort.enabled' as an example, there was a
> > > discussion[1] about it before.
> > >
> > > [1] https://lists.apache.org/thread/q5h3obx36pf9po28r0jzmwnmvtyjmwdr
> > >
> > > Lincoln Lee  于2024年5月20日周一 12:01写道:
> > > >
> > > > Hi Jane,
> > > >
> > > > Thanks for the proposal!
> > > >
> > > > +1 for the changes except for these annotated as experimental ones.
> > > >
> > > > For the options annotated as experimental,
> > > >
> > > > +1 for the moving of IncrementalAggregateRule & RelNodeBlock.
> > > >
> > > > For the rest of the options, there are some suggestions:
> > > >
> > > > 1. for the batch related parameters, it's recommended to either
> delete
> > > > them (leaving the necessary defaults value in place) or leave them as
> > > they
> > > > are. Including:
> > > > FlinkRelMdRowCount
> > > > FlinkRexUtil
> > > > BatchPhysicalSortRule
> > > > JoinDeriveNullFilterRule
> > > > BatchPhysicalJoinRuleBase
> > > > BatchPhysicalSortMergeJoinRule
> > > >
> > > > What I understand about the history of these options is that they
> were
> > > once
> > > > used for fine
> > > > tuning for tpc testing, and the current flink planner no longer
> relies
> > on
> > > > these internal
> > > > options when testing tpc[1]. In addition, these options are too
> obscure
> > > for
> > > > SQL users,
> > > > and some of them are actually magic numbers.
> > > >
> > > > 2. Regarding the options in HashAggCodeGenerator, since this new
> > feature
> > > > has gone
> > > > through a couple of release cycles and could be considered for
> > > > PublicEvolving now,
> > > > cc @Ron Liu   WDYT?
> > > >
> > > > 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on
> > TVF
> > > > window, so
> > > > it's recommended to keep it untouched for now and follow up in
> > > > FLINK-29692[2]. cc @Xuyang 
> > > >
> > > > [1]
> > > >
> > >
> >
> https://github.com/ververica/flink-sql-benchmark/blob/master/tools/common/flink-conf.yaml
> > > > [2] https://issues.apache.org/jira/browse/FLINK-29692
> > > >
> > > >
> > > > Best,
> > > > Lincoln Lee
> > > >
> > > >
> > > > Yubin Li  于2024年5月17日周五 10:49写道:
> > > >
> > > > > Hi Jane,
> > > > >
> > > > > Thank Jane for driving this proposal !
> > > > >
> > > > > This makes sense for users, +1 for that.
> > > > >
> > > > > Best,
> > > > > Yubin
> > > > >
> > > > > On Thu, May 16, 2024 at 3:17 PM Jark Wu  wrote:
> > > > > >
> > > > > > 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 think we
> > have
> > > to
> > > > > > > mark the old option deprecated in 1.20 for it to be removed in
> > 2.0,
> > > > > > > according to the API compatibility guarantees[1]. We can
> > introduce
> > > the
> > > > > > > new option in 1.20 with the same option key in the intended
> > class.
> > > > > > > WDY

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

2024-05-21 Thread Jane Chan
Hi all,

Thanks for your valuable feedback!

To @Xuannan

For options to be moved to another module/package, I think we have to
> mark the old option deprecated in 1.20 for it to be removed in 2.0,
> according to the API compatibility guarantees[1]. We can introduce the
> new option in 1.20 with the same option key in the intended class.


Good point, fixed.

To @Lincoln and @Benchao

Thanks for sharing the insights into the historical context of which I was
unaware. I've reorganized the sheet.

3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on TVF
> window, so it's recommended to keep it untouched for now and follow up in
> FLINK-29692


How to tackle the configuration is up to whether to remove the legacy
window aggregate in 2.0, and I've updated the FLIP to leverage this part to
FLINK-29692.

Please let me know if that answers your questions or if you have other
comments.

Best,
Jane


On Mon, May 20, 2024 at 1:52 PM Ron Liu  wrote:

> Hi, Lincoln
>
> >  2. Regarding the options in HashAggCodeGenerator, since this new feature
> has gone
> through a couple of release cycles and could be considered for
> PublicEvolving now,
> cc @Ron Liu   WDYT?
>
> Thanks for cc'ing me,  +1 for public these options now.
>
> Best,
> Ron
>
> Benchao Li  于2024年5月20日周一 13:08写道:
>
> > I agree with Lincoln about the experimental features.
> >
> > Some of these configurations do not even have proper implementation,
> > take 'table.exec.range-sort.enabled' as an example, there was a
> > discussion[1] about it before.
> >
> > [1] https://lists.apache.org/thread/q5h3obx36pf9po28r0jzmwnmvtyjmwdr
> >
> > Lincoln Lee  于2024年5月20日周一 12:01写道:
> > >
> > > Hi Jane,
> > >
> > > Thanks for the proposal!
> > >
> > > +1 for the changes except for these annotated as experimental ones.
> > >
> > > For the options annotated as experimental,
> > >
> > > +1 for the moving of IncrementalAggregateRule & RelNodeBlock.
> > >
> > > For the rest of the options, there are some suggestions:
> > >
> > > 1. for the batch related parameters, it's recommended to either delete
> > > them (leaving the necessary defaults value in place) or leave them as
> > they
> > > are. Including:
> > > FlinkRelMdRowCount
> > > FlinkRexUtil
> > > BatchPhysicalSortRule
> > > JoinDeriveNullFilterRule
> > > BatchPhysicalJoinRuleBase
> > > BatchPhysicalSortMergeJoinRule
> > >
> > > What I understand about the history of these options is that they were
> > once
> > > used for fine
> > > tuning for tpc testing, and the current flink planner no longer relies
> on
> > > these internal
> > > options when testing tpc[1]. In addition, these options are too obscure
> > for
> > > SQL users,
> > > and some of them are actually magic numbers.
> > >
> > > 2. Regarding the options in HashAggCodeGenerator, since this new
> feature
> > > has gone
> > > through a couple of release cycles and could be considered for
> > > PublicEvolving now,
> > > cc @Ron Liu   WDYT?
> > >
> > > 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on
> TVF
> > > window, so
> > > it's recommended to keep it untouched for now and follow up in
> > > FLINK-29692[2]. cc @Xuyang 
> > >
> > > [1]
> > >
> >
> https://github.com/ververica/flink-sql-benchmark/blob/master/tools/common/flink-conf.yaml
> > > [2] https://issues.apache.org/jira/browse/FLINK-29692
> > >
> > >
> > > Best,
> > > Lincoln Lee
> > >
> > >
> > > Yubin Li  于2024年5月17日周五 10:49写道:
> > >
> > > > Hi Jane,
> > > >
> > > > Thank Jane for driving this proposal !
> > > >
> > > > This makes sense for users, +1 for that.
> > > >
> > > > Best,
> > > > Yubin
> > > >
> > > > On Thu, May 16, 2024 at 3:17 PM Jark Wu  wrote:
> > > > >
> > > > > 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 think we
> have
> > to
> > > > > > mark the old option deprecated in 1.20 for it to be removed in
> 2.0,
> > > > > > according to the API compatibility guarantees[1]. We can
> introduce
> > the
> > > > > > new option in 1.20 with the same option key in the intended
> class.
> > > > > > WDYT?
> > > > > >
> > > > > > Best,
> > > > > > Xuannan
> > > > > >
> > > > > > [1]
> > > > > >
> > > >
> >
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Wed, May 15, 2024 at 6:20 PM Jane Chan  >
> > > > wrote:
> > > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > I'd like to start a discussion on FLIP-457: Improve Table/SQL
> > > > > > Configuration
> > > > > > > for Flink 2.0 [1]. This FLIP revisited all Table/SQL
> > configurations
> > > > to
> 

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

2024-05-19 Thread Ron Liu
Hi, Lincoln

>  2. Regarding the options in HashAggCodeGenerator, since this new feature
has gone
through a couple of release cycles and could be considered for
PublicEvolving now,
cc @Ron Liu   WDYT?

Thanks for cc'ing me,  +1 for public these options now.

Best,
Ron

Benchao Li  于2024年5月20日周一 13:08写道:

> I agree with Lincoln about the experimental features.
>
> Some of these configurations do not even have proper implementation,
> take 'table.exec.range-sort.enabled' as an example, there was a
> discussion[1] about it before.
>
> [1] https://lists.apache.org/thread/q5h3obx36pf9po28r0jzmwnmvtyjmwdr
>
> Lincoln Lee  于2024年5月20日周一 12:01写道:
> >
> > Hi Jane,
> >
> > Thanks for the proposal!
> >
> > +1 for the changes except for these annotated as experimental ones.
> >
> > For the options annotated as experimental,
> >
> > +1 for the moving of IncrementalAggregateRule & RelNodeBlock.
> >
> > For the rest of the options, there are some suggestions:
> >
> > 1. for the batch related parameters, it's recommended to either delete
> > them (leaving the necessary defaults value in place) or leave them as
> they
> > are. Including:
> > FlinkRelMdRowCount
> > FlinkRexUtil
> > BatchPhysicalSortRule
> > JoinDeriveNullFilterRule
> > BatchPhysicalJoinRuleBase
> > BatchPhysicalSortMergeJoinRule
> >
> > What I understand about the history of these options is that they were
> once
> > used for fine
> > tuning for tpc testing, and the current flink planner no longer relies on
> > these internal
> > options when testing tpc[1]. In addition, these options are too obscure
> for
> > SQL users,
> > and some of them are actually magic numbers.
> >
> > 2. Regarding the options in HashAggCodeGenerator, since this new feature
> > has gone
> > through a couple of release cycles and could be considered for
> > PublicEvolving now,
> > cc @Ron Liu   WDYT?
> >
> > 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on TVF
> > window, so
> > it's recommended to keep it untouched for now and follow up in
> > FLINK-29692[2]. cc @Xuyang 
> >
> > [1]
> >
> https://github.com/ververica/flink-sql-benchmark/blob/master/tools/common/flink-conf.yaml
> > [2] https://issues.apache.org/jira/browse/FLINK-29692
> >
> >
> > Best,
> > Lincoln Lee
> >
> >
> > Yubin Li  于2024年5月17日周五 10:49写道:
> >
> > > Hi Jane,
> > >
> > > Thank Jane for driving this proposal !
> > >
> > > This makes sense for users, +1 for that.
> > >
> > > Best,
> > > Yubin
> > >
> > > On Thu, May 16, 2024 at 3:17 PM Jark Wu  wrote:
> > > >
> > > > 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 think we have
> to
> > > > > mark the old option deprecated in 1.20 for it to be removed in 2.0,
> > > > > according to the API compatibility guarantees[1]. We can introduce
> the
> > > > > new option in 1.20 with the same option key in the intended class.
> > > > > WDYT?
> > > > >
> > > > > Best,
> > > > > Xuannan
> > > > >
> > > > > [1]
> > > > >
> > >
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees
> > > > >
> > > > >
> > > > >
> > > > > On Wed, May 15, 2024 at 6:20 PM Jane Chan 
> > > wrote:
> > > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > I'd like to start a discussion on FLIP-457: Improve Table/SQL
> > > > > Configuration
> > > > > > for Flink 2.0 [1]. This FLIP revisited all Table/SQL
> configurations
> > > to
> > > > > > improve user-friendliness and maintainability as Flink moves
> toward
> > > 2.0.
> > > > > >
> > > > > > I am looking forward to your feedback.
> > > > > >
> > > > > > Best regards,
> > > > > > Jane
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=307136992
> > > > >
> > >
>
>
>
> --
>
> Best,
> Benchao Li
>


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

2024-05-19 Thread Benchao Li
I agree with Lincoln about the experimental features.

Some of these configurations do not even have proper implementation,
take 'table.exec.range-sort.enabled' as an example, there was a
discussion[1] about it before.

[1] https://lists.apache.org/thread/q5h3obx36pf9po28r0jzmwnmvtyjmwdr

Lincoln Lee  于2024年5月20日周一 12:01写道:
>
> Hi Jane,
>
> Thanks for the proposal!
>
> +1 for the changes except for these annotated as experimental ones.
>
> For the options annotated as experimental,
>
> +1 for the moving of IncrementalAggregateRule & RelNodeBlock.
>
> For the rest of the options, there are some suggestions:
>
> 1. for the batch related parameters, it's recommended to either delete
> them (leaving the necessary defaults value in place) or leave them as they
> are. Including:
> FlinkRelMdRowCount
> FlinkRexUtil
> BatchPhysicalSortRule
> JoinDeriveNullFilterRule
> BatchPhysicalJoinRuleBase
> BatchPhysicalSortMergeJoinRule
>
> What I understand about the history of these options is that they were once
> used for fine
> tuning for tpc testing, and the current flink planner no longer relies on
> these internal
> options when testing tpc[1]. In addition, these options are too obscure for
> SQL users,
> and some of them are actually magic numbers.
>
> 2. Regarding the options in HashAggCodeGenerator, since this new feature
> has gone
> through a couple of release cycles and could be considered for
> PublicEvolving now,
> cc @Ron Liu   WDYT?
>
> 3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on TVF
> window, so
> it's recommended to keep it untouched for now and follow up in
> FLINK-29692[2]. cc @Xuyang 
>
> [1]
> https://github.com/ververica/flink-sql-benchmark/blob/master/tools/common/flink-conf.yaml
> [2] https://issues.apache.org/jira/browse/FLINK-29692
>
>
> Best,
> Lincoln Lee
>
>
> Yubin Li  于2024年5月17日周五 10:49写道:
>
> > Hi Jane,
> >
> > Thank Jane for driving this proposal !
> >
> > This makes sense for users, +1 for that.
> >
> > Best,
> > Yubin
> >
> > On Thu, May 16, 2024 at 3:17 PM Jark Wu  wrote:
> > >
> > > 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 think we have to
> > > > mark the old option deprecated in 1.20 for it to be removed in 2.0,
> > > > according to the API compatibility guarantees[1]. We can introduce the
> > > > new option in 1.20 with the same option key in the intended class.
> > > > WDYT?
> > > >
> > > > Best,
> > > > Xuannan
> > > >
> > > > [1]
> > > >
> > https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees
> > > >
> > > >
> > > >
> > > > On Wed, May 15, 2024 at 6:20 PM Jane Chan 
> > wrote:
> > > > >
> > > > > Hi all,
> > > > >
> > > > > I'd like to start a discussion on FLIP-457: Improve Table/SQL
> > > > Configuration
> > > > > for Flink 2.0 [1]. This FLIP revisited all Table/SQL configurations
> > to
> > > > > improve user-friendliness and maintainability as Flink moves toward
> > 2.0.
> > > > >
> > > > > I am looking forward to your feedback.
> > > > >
> > > > > Best regards,
> > > > > Jane
> > > > >
> > > > > [1]
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=307136992
> > > >
> >



-- 

Best,
Benchao Li


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

2024-05-19 Thread Lincoln Lee
Hi Jane,

Thanks for the proposal!

+1 for the changes except for these annotated as experimental ones.

For the options annotated as experimental,

+1 for the moving of IncrementalAggregateRule & RelNodeBlock.

For the rest of the options, there are some suggestions:

1. for the batch related parameters, it's recommended to either delete
them (leaving the necessary defaults value in place) or leave them as they
are. Including:
FlinkRelMdRowCount
FlinkRexUtil
BatchPhysicalSortRule
JoinDeriveNullFilterRule
BatchPhysicalJoinRuleBase
BatchPhysicalSortMergeJoinRule

What I understand about the history of these options is that they were once
used for fine
tuning for tpc testing, and the current flink planner no longer relies on
these internal
options when testing tpc[1]. In addition, these options are too obscure for
SQL users,
and some of them are actually magic numbers.

2. Regarding the options in HashAggCodeGenerator, since this new feature
has gone
through a couple of release cycles and could be considered for
PublicEvolving now,
cc @Ron Liu   WDYT?

3. Regarding WindowEmitStrategy, IIUC it is currently unsupported on TVF
window, so
it's recommended to keep it untouched for now and follow up in
FLINK-29692[2]. cc @Xuyang 

[1]
https://github.com/ververica/flink-sql-benchmark/blob/master/tools/common/flink-conf.yaml
[2] https://issues.apache.org/jira/browse/FLINK-29692


Best,
Lincoln Lee


Yubin Li  于2024年5月17日周五 10:49写道:

> Hi Jane,
>
> Thank Jane for driving this proposal !
>
> This makes sense for users, +1 for that.
>
> Best,
> Yubin
>
> On Thu, May 16, 2024 at 3:17 PM Jark Wu  wrote:
> >
> > 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 think we have to
> > > mark the old option deprecated in 1.20 for it to be removed in 2.0,
> > > according to the API compatibility guarantees[1]. We can introduce the
> > > new option in 1.20 with the same option key in the intended class.
> > > WDYT?
> > >
> > > Best,
> > > Xuannan
> > >
> > > [1]
> > >
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees
> > >
> > >
> > >
> > > On Wed, May 15, 2024 at 6:20 PM Jane Chan 
> wrote:
> > > >
> > > > Hi all,
> > > >
> > > > I'd like to start a discussion on FLIP-457: Improve Table/SQL
> > > Configuration
> > > > for Flink 2.0 [1]. This FLIP revisited all Table/SQL configurations
> to
> > > > improve user-friendliness and maintainability as Flink moves toward
> 2.0.
> > > >
> > > > I am looking forward to your feedback.
> > > >
> > > > Best regards,
> > > > Jane
> > > >
> > > > [1]
> > > >
> > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=307136992
> > >
>


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

2024-05-16 Thread Yubin Li
Hi Jane,

Thank Jane for driving this proposal !

This makes sense for users, +1 for that.

Best,
Yubin

On Thu, May 16, 2024 at 3:17 PM Jark Wu  wrote:
>
> 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 think we have to
> > mark the old option deprecated in 1.20 for it to be removed in 2.0,
> > according to the API compatibility guarantees[1]. We can introduce the
> > new option in 1.20 with the same option key in the intended class.
> > WDYT?
> >
> > Best,
> > Xuannan
> >
> > [1]
> > https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees
> >
> >
> >
> > On Wed, May 15, 2024 at 6:20 PM Jane Chan  wrote:
> > >
> > > Hi all,
> > >
> > > I'd like to start a discussion on FLIP-457: Improve Table/SQL
> > Configuration
> > > for Flink 2.0 [1]. This FLIP revisited all Table/SQL configurations to
> > > improve user-friendliness and maintainability as Flink moves toward 2.0.
> > >
> > > I am looking forward to your feedback.
> > >
> > > Best regards,
> > > Jane
> > >
> > > [1]
> > >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=307136992
> >


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 think we have to
> mark the old option deprecated in 1.20 for it to be removed in 2.0,
> according to the API compatibility guarantees[1]. We can introduce the
> new option in 1.20 with the same option key in the intended class.
> WDYT?
>
> Best,
> Xuannan
>
> [1]
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees
>
>
>
> On Wed, May 15, 2024 at 6:20 PM Jane Chan  wrote:
> >
> > Hi all,
> >
> > I'd like to start a discussion on FLIP-457: Improve Table/SQL
> Configuration
> > for Flink 2.0 [1]. This FLIP revisited all Table/SQL configurations to
> > improve user-friendliness and maintainability as Flink moves toward 2.0.
> >
> > I am looking forward to your feedback.
> >
> > Best regards,
> > Jane
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=307136992
>


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

2024-05-15 Thread Xuannan Su
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 think we have to
mark the old option deprecated in 1.20 for it to be removed in 2.0,
according to the API compatibility guarantees[1]. We can introduce the
new option in 1.20 with the same option key in the intended class.
WDYT?

Best,
Xuannan

[1] 
https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees



On Wed, May 15, 2024 at 6:20 PM Jane Chan  wrote:
>
> Hi all,
>
> I'd like to start a discussion on FLIP-457: Improve Table/SQL Configuration
> for Flink 2.0 [1]. This FLIP revisited all Table/SQL configurations to
> improve user-friendliness and maintainability as Flink moves toward 2.0.
>
> I am looking forward to your feedback.
>
> Best regards,
> Jane
>
> [1]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=307136992