Hi, xuyang Thanks for starting this FLIP discussion, currently there are two types of window aggregation in Flink SQL, namely legacy group window aggregation and window tvf aggregation, these two types of window aggregation are not fully aligned in behavior, which will bring a lot of confusion to the users, so there is a need to unify and align them. I think the final ideal state should be that there is only one window tvf aggregation, which supports Tumble, HOP, Cumulate and Session windows, and supports consuming CDC data streams. There is also support for configuring EARLY-FIRE and LATER-FIRE.
This FLIP is a continuation of FLIP-145, and also supports legacy group window aggregation to flat-migrate to the new window tvf agregation, which is very useful, especially for the support of CDC streams, a pain point that users often feedback. Big +1 for this FLIP. Best, Ron Xuyang <xyzhong...@163.com> 于2023年12月5日周二 11:11写道: > Hi, Feng and David. > > > Thank you very much to share your thoughts. > > > This flip does not include the official exposure of these experimental > conf to users. Thus there is not adetailed description of this part. > However, in view that some technical users may have added these > experimental conf in actual production jobs, the processing > of these conf while using window tvf syntax has been added to this flip. > > > Overall, the behavior of using these experimental parameters is no > different from before, and I think we should provide the compatibility > about using these experimental conf. > > > Look for your thoughs. > > > > > -- > > Best! > Xuyang > > > > > > At 2023-12-05 09:17:49, "David Anderson" <dander...@apache.org> wrote: > >The current situation (where we have both the legacy windows and the > >TVF-based windows) is confusing for users, and I'd like to see us move > >forward as rapidly as possible. > > > >Since the early fire, late fire, and allowed lateness features were never > >documented or exposed to users, I don't feel that we need to provide > >replacements for these internal, experimental features before officially > >deprecating the legacy group window aggregations, and I'd rather not wait. > > > >However, I'd be delighted to see a proposal for what that might look like. > > > >Best, > >David > > > >On Mon, Dec 4, 2023 at 12:45 PM Feng Jin <jinfeng1...@gmail.com> wrote: > > > >> Hi xuyang, > >> > >> Thank you for initiating this proposal. > >> > >> I'm glad to see that TVF's functionality can be fully supported. > >> > >> Regarding the early fire, late fire, and allow lateness features, how > will > >> they be provided to users? The documentation doesn't seem to provide a > >> detailed description of this part. > >> > >> Since this FLIP will also involve a lot of feature development, I am > more > >> than willing to help, including development and code review. > >> > >> Best, > >> Feng > >> > >> On Tue, Nov 28, 2023 at 8:31 PM 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 > >> >