Thanks Martijn and Xuyang, I opened a new discussion thread about FLIP-377. https://lists.apache.org/thread/nvxx8sp9jm009yywm075hoffr632tm7j
Best, Jiabao > 2023年10月24日 15:39,Martijn Visser <martijnvis...@apache.org> 写道: > > Hi, > > Please convert the Google Doc into a FLIP and start a FLIP discussion. > > Best regards, > > Martijn > > On Tue, Oct 24, 2023 at 9:20 AM Xuyang <xyzhong...@163.com> wrote: >> >> +1. How to set the configuration value so that the specific source can be >> perceived needs to be considered. >> >> >> >> >> -- >> >> Best! >> Xuyang >> >> >> >> >> >> At 2023-10-24 15:05:03, "Jiabao Sun" <jiabao....@xtransfer.cn.INVALID> wrote: >>> Thanks Xuyang, >>> >>> 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 applyFilters >>> method to determine whether filter pushdown is needed. >>> This would increase complexity and violate the original behavior of the >>> applyFilters method. >>> >>> On the contrary, we only need to pass the configuration parameter in the >>> newly added enableFilterPushDown method >>> to decide whether to perform predicate pushdown. >>> >>> I think this approach would be clearer and simpler. >>> >>> Best, >>> Jiabao >>> >>> >>>> 2023年10月24日 14:34,Jiabao Sun <jiabao....@xtransfer.cn.INVALID> 写道: >>>> >>>> Thanks Xuyang, >>>> >>>> The table.optimizer.source.predicate-pushdown-enabled options do not >>>> provide fine-grained configuration for each source. >>>> >>>> Suppose we have an SQL query with two sources: Kafka and a database (CDC). >>>> The database is sensitive to pressure, and we want to configure it to not >>>> perform filter pushdown to the database source. >>>> However, we still want to perform filter pushdown to the Kafka source to >>>> decrease network IO. >>>> >>>> >>>> Best, >>>> Jiabao >>>> <https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/dev/table/config/#table-optimizer-source-predicate-pushdown-enabled> >>>> >>>>> 2023年10月24日 14:24,Xuyang <xyzhong...@163.com> 写道: >>>>> >>>>> Hi, the existant configuration >>>>> 'table.optimizer.source.predicate-pushdown-enabled' seems to do what you >>>>> want. >>>>> Can you describe more clearly the difference between what you want and >>>>> this configuration ? >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> -- >>>>> >>>>> Best! >>>>> Xuyang >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> At 2023-10-24 14:12:14, "Jiabao Sun" <jiabao....@xtransfer.cn.INVALID> >>>>> wrote: >>>>>> Hi Devs, >>>>>> >>>>>> I would like to start a discussion on support configuration to disable >>>>>> filter pushdown for Table/SQL Sources[1]. >>>>>> >>>>>> Currently, Flink SQL does not support the ability for users to enable or >>>>>> disable filter pushdown. >>>>>> However, filter pushdown has some side effects, such as additional >>>>>> computational pressure on external systems. >>>>>> Moreover, Improper queries can lead to issues such as full table scans, >>>>>> which in turn can impact the stability of external systems. >>>>>> >>>>>> I propose to support configuration to disable filter push down for >>>>>> Table/SQL sources to let user decide whether to perform filter pushdown. >>>>>> >>>>>> Looking forward to your feedback. >>>>>> >>>>>> [1] >>>>>> https://docs.google.com/document/d/1QsbOi9InvmfwFr8YbrnnXOKLPnb8JnqhXIMbGd68SFU/edit?usp=sharing >>>>>> >>>>>> Best, >>>>>> Jiabao