Hi Xuyang, As a clarification, since FLINK-24204 is finishing up work from FLIP-145[1], do we need to discuss anything before you work out the details of FLINK-24024 as a PR?
Relatedly, as that goes up for a PR, as part of FLINK-33421 [2], Bonnie and I are working through migrating some of the JsonPlan Tests and ITCases to RestoreTests. I've got a PR up [3] for moving at least one of the classes you are touching. Let me know if I can share any details about that work. Cheers, Jim 1. https://cwiki.apache.org/confluence/display/FLINK/FLIP-145%3A+Support+SQL+windowing+table-valued+function#FLIP145:SupportSQLwindowingtablevaluedfunction-SessionWindows 2. https://issues.apache.org/jira/browse/FLINK-33421 3. https://github.com/apache/flink/pull/23886 https://issues.apache.org/jira/browse/FLINK-33676 On Tue, Nov 28, 2023 at 7:31 AM Xuyang <xyzhong...@163.com> wrote: > Hi all. > I'd like to start a discussion of FLIP-392: Deprecate the Legacy Group > Window Aggregation. > > > Although the current Flink SQL Window Aggregation documentation[1] > indicates that the legacy Group Window Aggregation > syntax has been deprecated, the new Window TVF Aggregation syntax has not > fully covered all of the features of the legacy one. > > > Compared to Group Window Aggergation, Window TVF Aggergation has several > advantages, such as two-stage optimization, > support for standard GROUPING SET syntax, and so on. However, it needs to > supplement and enrich the following features. > > > 1. Support for SESSION Window TVF Aggregation > 2. Support for consuming CDC stream > 3. Support for HOP window size with non-integer step length > 4. Support for configurations such as early fire, late fire and allow > lateness > (which are internal experimental configurations in Group Window > Aggregation and not public to users yet.) > 5. Unification of the Window TVF Aggregation operator in runtime at the > implementation layer > (In the long term, the cost to maintain the operators about Window TVF > Aggregation and Group Window Aggregation is too expensive.) > > > This flip aims to continue the unfinished work in FLIP-145[2], which is to > fully enable the capabilities of Window TVF Aggregation > and officially deprecate the legacy syntax Group Window Aggregation, to > prepare for the removal of the legacy one in Flink 2.0. > > > I have already done some preliminary POC to validate the feasibility of > the related work in this flip as follows. > 1. POC for SESSION Window TVF Aggregation [3] > 2. POC for CUMULATE in Group Window Aggregation operator [4] > 3. POC for consuming CDC stream in Window Aggregation operator [5] > > > Looking forward to your feedback and thoughts! > > > > [1] > https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sql/queries/window-agg/ > > [2] > https://cwiki.apache.org/confluence/display/FLINK/FLIP-145%3A+Support+SQL+windowing+table-valued+function#FLIP145:SupportSQLwindowingtablevaluedfunction-SessionWindows > [3] https://github.com/xuyangzhong/flink/tree/FLINK-24024 > [4] > https://github.com/xuyangzhong/flink/tree/poc_legacy_group_window_agg_cumulate > [5] > https://github.com/xuyangzhong/flink/tree/poc_window_agg_consumes_cdc_stream > > > > -- > > Best! > Xuyang