Thanks Leonard for your reminder,
The FLIP title has been changed as FLIP-377:
Support fine-grained configuration to control filter push down for Table/SQL
Sources.
Best,
Jiabao
On 2024/01/03 06:51:10 Leonard Xu wrote:
> Thanks Jiabao for driving this.
>
> +1 to start a vote, a minor
Thanks Jiabao for driving this.
+1 to start a vote, a minor comment, should we change the FLIP title according
this thread context as well?
Best,
Leonard
> 2024年1月3日 下午2:43,Jiabao Sun 写道:
>
> Hi,
>
> Thank you again for the discussion on this FLIP.
> If there are no further comments, I
Hi,
Thank you again for the discussion on this FLIP.
If there are no further comments, I plan to start a voting thread tomorrow.
Best,
Jiabao
On 2023/12/20 14:09:49 Jiabao Sun wrote:
> Hi,
>
> Thank you to everyone for the discussion on this FLIP,
> especially Becket for providing guidance
Hi,
Thank you to everyone for the discussion on this FLIP,
especially Becket for providing guidance that made it more reasonable.
The FLIP document[1] has been updated with the recent discussed content.
Please take a look to double-check it when you have time.
If we can reach a consensus on
Thanks Becket,
The behavior description has been added to the Public Interfaces section.
Best,
Jiabao
> 2023年12月20日 08:17,Becket Qin 写道:
>
> Hi Jiabao,
>
> Thanks for updating the FLIP.
> Can you add the behavior of the policies that are only applicable to some
> but not all of the
Hi Jiabao,
Thanks for updating the FLIP.
Can you add the behavior of the policies that are only applicable to some
but not all of the databases? This is a part of the intended behavior of
the proposed configuration. So, we should include that in the FLIP.
Otherwise, the FLIP looks good to me.
Hi Becket,
I share the same view as you regarding the prefix for this configuration option.
For the JDBC connector, I prefer setting 'filter.handling.policy' = 'FOO' and
throwing an exception when the database do not support that specific policy.
Not using a prefix can reduce the learning
Hi David,
Sorry, the last two comments at the bottom of the email list were replies from
a long time ago.
I didn't have a good understanding of the display order of the email list
before, which caused inconvenience.
You can refer to the comments above and the latest FLIP-377 document[1] for
Hi,
I had a 3 of comments:
- the name of the config option is "scan.filter-push-down.enabled". This
implies it is only for scan sources and not lookups. I suggest removing the
scan. prefix.
- there is a talk of having a numeric option, as the filter pushdown might
result in a full table scan.
Hi Jiabao,
Thanks for updating the FLIP.
One more question regarding the JDBC connector, since it is a connector
shared by multiple databases, what if there is a filter handling policy
that is only applicable to one of the databases, but not the others? In
that case, how would the users specify
Thanks Becket for the suggestions,
Updated.
Please help review it again when you have time.
Best,
Jiabao
> 2023年12月19日 09:06,Becket Qin 写道:
>
> Hi JIabao,
>
> Thanks for updating the FLIP. It looks better. Some suggestions / questions:
>
> 1. In the motivation section:
>
>> *Currently,
Hi JIabao,
Thanks for updating the FLIP. It looks better. Some suggestions / questions:
1. In the motivation section:
> *Currently, Flink Table/SQL does not expose fine-grained control for users
> to control filter pushdown. **However, filter pushdown has some side
> effects, such as additional
Hi Becket,
The FLIP document[1] has been updated.
Could you help take a look again?
Thanks,
Jiabao
[1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=276105768
> 2023年12月18日 16:53,Becket Qin 写道:
>
> Yes, that sounds reasonable to me. We can start with ALWAYS and NEVER, and
Yes, that sounds reasonable to me. We can start with ALWAYS and NEVER, and
add more policies as needed.
Thanks,
Jiangjie (Becket) Qin
On Mon, Dec 18, 2023 at 4:48 PM Jiabao Sun
wrote:
> Thanks Bucket,
>
> The jdbc.filter.handling.policy is good to me as it provides sufficient
> extensibility
Thanks Bucket,
The jdbc.filter.handling.policy is good to me as it provides sufficient
extensibility for future filter pushdown optimizations.
However, currently, we don't have an implementation for the AUTO mode, and it
seems that the AUTO mode can easily be confused with the ALWAYS mode
Hi Jiabao,
Please see the reply inline.
> The MySQL connector is currently in the flink-connector-jdbc repository
> and is not a standalone connector.
> Is it too unique to use "mysql" as the configuration option prefix?
If the intended behavior makes sense to all the supported JDBC drivers,
Hi Becket,
The MySQL connector is currently in the flink-connector-jdbc repository and is
not a standalone connector.
Is it too unique to use "mysql" as the configuration option prefix?
Also, I would like to ask about the difference in behavior between AUTO and
ALWAYS.
It seems that we
Hi JIabao,
Thanks for updating the FLIP. Maybe I did not explain it clearly enough. My
point is that given there are various good flavors of behaviors handling
filters pushed down, we should not have a common config of
"ignore.filter.pushdown", because the behavior is not *common*.
It looks like
Hi Becket,
The FLIP document has been updated as well.
Please take a look when you have time.
Thanks,
Jiabao
> 2023年12月17日 22:54,Jiabao Sun 写道:
>
> Thanks Becket,
>
> I apologize for not being able to continue with this proposal due to being
> too busy during this period.
>
> The
Thanks Becket,
I apologize for not being able to continue with this proposal due to being too
busy during this period.
The viewpoints you shared about the design of Flink Source make sense to me
The native configuration ‘ignore.filter.pushdown’ is good to me.
Having a unified name or naming
Hi Jiabao,
Arguments like "because Spark has it so Flink should also have it" does not
make sense. Different projects have different API flavors and styles. What
is really important is the rationale and the design principle behind the
API. They should conform to the convention of the project.
Thanks Becket,
I still believe that adding a configuration at the source level to disable
filter pushdown is needed. This demand exists in spark as well[1].
In Spark, most sources that support filter pushdown provide their own
corresponding configuration options to enable or disable filter
Hi Jiabao,
While we can always fix the formality of the config, a more fundamental
issue here is whether this configuration is common enough. Personally I am
still not convinced it is.
Remember we don't have a common implementation for SupportsFilterPushdown
itself. Why does a potential behavior
Thanks Becket for the feedback,
Regarding concerns about common configurations, I think we can introduce
FiltersApplier to unify the behavior of various connectors.
public static class FiltersApplier {
private final ReadableConfig config;
private final Function, Result> action;
Hi Jiabao,
Yes, I still have concerns.
The FLIP violates the following two principles regarding configuration:
1.* A config of a class should never negate the semantic of a decorative
interface implemented by that class. *
A decorative interface is a public contract with other components, while
Hi Becket,
The purpose of introducing this configuration is that not all filter pushdowns
can improve overall performance.
If the filter can hit the external index, then pushdown is definitely worth it,
as it can not only improve query time but also decrease network overhead.
However, for
Hi Becket,
Actually, for FileSystemSource, it is not always desired, only OCR file formats
support filter pushdown.
We can disable predicate pushdown for FileSystemSource by setting
'table.optimizer.source.predicate-pushdown-enabled' to false.
I think we can also disable filter pushdown at a
Hi Jiabao,
Thanks for the explanation. Maybe it's easier to explain with an example.
Let's take FileSystemTableSource as an example. Currently it implements
SupportsFilterPushDown interface. With your proposal, does it have to
support `source.filter-push-down.enabled` as well? But this
Thanks Becket for the further explanation.
Perhaps I didn't explain it clearly.
1. If a source does not implement the SupportsFilterPushDown interface, the
newly added configurations do not need to be added to either the
requiredOptions or optionalOptions.
Similar to LookupOptions, if a
Hi Jiabao,
Please see the replies inline.
Introducing common configurations does not mean that all sources must
> accept these configuration options.
> The configuration options supported by a source are determined by the
> requiredOptions and optionalOptions in the Factory interface.
This is
Hi Becket,
Sorry, there was a typo in the second point. Let me correct it:
Introducing common configurations does not mean that all sources must accept
these configuration options.
The configuration options supported by a source are determined by the
requiredOptions and optionalOptions in
Thanks Becket for the feedback.
1. Currently, the SupportsFilterPushDown#applyFilters method returns a result
that includes acceptedFilters and remainingFilters. The source can decide to
push down some filters or not accept any of them.
2. Introducing common configuration options does not mean
Hi Jiabao and Ruanhang,
Adding a configuration of source.filter-push-down.enabled as a common
source configuration seems problematic.
1. The config name is misleading. filter pushdown should only be determined
by whether the SupportsFilterPushdown interface is implemented or not.
2. The behavior
Thanks Hang for the suggestion.
I think the configuration of TableSource is not closely related to
SourceReader,
so I prefer to introduce a independent configuration class TableSourceOptions
in the flink-table-common module, similar to LookupOptions.
For the second point, I suggest adding
Thanks for the improvements, Jiabao.
There are some details that I am not sure about.
1. The new option `source.filter-push-down.enabled` will be added to which
class? I think it should be `SourceReaderOptions`.
2. How are the connector developers able to know and follow the FLIP? Do we
need an
Hi, all,
Thanks for the lively discussion.
Based on the discussion, I have made some adjustments to the FLIP document:
1. The name of the newly added option has been changed to
"source.filter-push-down.enabled".
2. Considering compatibility with older versions, the newly added
Thanks for digging into the git history, Jark. I agree it makes sense to
deprecate this API in 2.0.
Cheers,
Jiangjie (Becket) Qin
On Fri, Oct 27, 2023 at 5:47 PM Jark Wu wrote:
> Hi Becket,
>
> I checked the history of "
> *table.optimizer.source.predicate-pushdown-enabled*",
> it seems it
Thanks Venkatakrishnan for the feedback.
Taking MySQL as an example, if the pushed-down filter does not hit an index, it
will result in a full table scan.
For a table with a large amount of data, a full table scan can consume a
significant amount of CPU resources,
increase response time, hold
Thanks for the proposal, Jiabao.
I agree with Becket if a *Source* is implementing the *SupportsXXXPushDown*
(in this case *SupportsFilterPushdown*) interface, then the *Source* (in
your FLIP example which is a database) is designed to support filter
pushdown. The corresponding Source can have
Hi Becket,
I checked the history of "
*table.optimizer.source.predicate-pushdown-enabled*",
it seems it was introduced since the legacy FilterableTableSource interface
which might be an experiential feature at that time. I don't see the
necessity
of this option at the moment. Maybe we can
Thanks for the proposal, Jiabao. My two cents below:
1. If I understand correctly, the motivation of the FLIP is mainly to make
predicate pushdown optional on SOME of the Sources. If so, intuitively the
configuration should be Source specific instead of general. Otherwise, we
will end up with
Thanks Jane for the detailed explanation.
I think that for users, we should respect conventions over configurations.
Conventions can be default values explicitly specified in configurations, or
they can be behaviors that follow previous versions.
If the same code has different behaviors in
Hi Jiabao,
Thanks for the in-depth clarification. Here are my cents
However, "table.optimizer.source.predicate-pushdown-enabled" and
> "scan.filter-push-down.enabled" are configurations for different
> components(optimizer and source operator).
>
We cannot assume that every user would be
Thanks Hang and Lincoln for the good point.
'source.predicate-pushdown.enabled’ is great to me. I have changed the proposal
document.
Do we need to maintain consistency in hyphen-separated naming style between
'source.predicate-pushdown-enabled' and
Hi, all,
Thanks for the lively discussion.
I agree with Jiabao. I think enabling "scan.filter-push-down.enabled"
relies on enabling "table.optimizer.source.predicate-pushdown-enabled".
It is a little strange that the planner still needs to push down the
filters when we set
Thanks Benchao for the feedback.
I understand that the configuration of global parallelism and task parallelism
is at different granularities but with the same configuration.
However, "table.optimizer.source.predicate-pushdown-enabled" and
"scan.filter-push-down.enabled" are configurations for
Thank you all for the lively discussion!
Agree with Benchao that from a user's (rather than a developer's) point of
view, it's easier to understand that fine-grained options override global
options.
In addition, for the new option 'scan.filter-push-down.enabled', would it
be
better to keep the
Thanks Jiabao for the detailed explanations, that helps a lot, I
understand your rationale now.
Correct me if I'm wrong. Your perspective is from "developer", which
means there is an optimizer and connector component, and if we want to
enable this feature (pushing filters down into connectors),
Thanks Jane for further explanation.
These two configurations correspond to different levels.
"scan.filter-push-down.enabled" does not make
"table.optimizer.source.predicate" invalid.
The planner will still push down predicates to all sources.
Whether filter pushdown is allowed or not is
Hi Jiabao,
Thanks for clarifying this. While by "scan.filter-push-down.enabled takes a
higher priority" I meant that this value should be respected whenever it is
set explicitly.
The conclusion that
2. "table.optimizer.source.predicate" = "true" and
> "scan.filter-push-down.enabled" = "false"
>
Thanks Benchao for the feedback.
For the current proposal, we recommend keeping the default value of
"table.optimizer.source.predicate" as true,
and setting the the default value of newly introduced option
"scan.filter-push-down.enabled" to true as well.
The main purpose of doing this is to
I agree with Jane that fine-grained configurations should have higher
priority than job level configurations.
For current proposal, we can achieve that:
- Set "table.optimizer.source.predicate" = "true" to enable by
default, and set ""scan.filter-push-down.enabled" = "false" to disable
it per
+1 for this FLIP.
BTW, I think we can add an option for projection push down too.
Yes, we can do all things in the connector, but a common
implementation helps a lot! And can introduce an unify option!
Best,
Jingsong
On Wed, Oct 25, 2023 at 10:07 AM Jark Wu wrote:
>
> Thank you for updating
Thank you for updating Jiabao,
The FLIP looks good to me.
Best,
Jark
On Wed, 25 Oct 2023 at 00:42, Jiabao Sun
wrote:
> Thanks Jane for the feedback.
>
> The default value of "table.optimizer.source.predicate" is true that means
> by default,
> allowing predicate pushdown to all sources is
Thanks Jane for the feedback.
The default value of "table.optimizer.source.predicate" is true that means by
default,
allowing predicate pushdown to all sources is permitted.
Therefore, disabling the pushdown filter for individual sources can take effect.
Best,
Jiabao
> 2023年10月24日
>
> I believe that the configuration "table.optimizer.source.predicate" has a
> higher priority at the planner level than the configuration at the source
> level,
> and it seems easy to implement now.
>
Correct me if I'm wrong, but I think the fine-grained configuration
Thanks Jark, Martijn, Xuyang for the valuable feedback.
Adding only the "scan.filter-push-down.enabled" configuration option would be
great for me as well.
Optimization for this public behavior can be added later.
I made some modifications to the FLIP document and added the approach of adding
Hi Jiabao,
I don't see that as a concern, but something that would be in general
preferred (because it gives more flexibility to users when to enable /
disable pushdown).
Best regards,
Martijn
On Tue, Oct 24, 2023 at 1:41 PM Hang Ruan wrote:
>
> Hi, Jiabao.
>
> Thanks for driving this
Hi, Jiabao.
Thanks for driving this discussion.
IMO, if there are many connectors containing the same logic, I think this
FLIP is useful.
We do not know how many connectors need to add the same code.
Best,
Hang
Jiabao Sun 于2023年10月24日周二 18:26写道:
> Thanks Martijn,
>
> Indeed, implementing the
Thanks Martijn,
Indeed, implementing the logic check in the applyFilters method can fulfill the
functionality of disabling filter pushdown.
My concern is that the same logic check may need to be implemented in each
source.
public Result applyFilters(List filters) {
if
Just like our discussion in thread
https://lists.apache.org/thread/whh75f6rtwdyqxt47gb39j6m6m0cpphq , +1 for this
Flip.
--
Best!
Xuyang
在 2023-10-24 18:03:36,"Jiabao Sun" 写道:
>Thanks Jane,
>
>I believe that the configuration "table.optimizer.source.predicate" has a
>higher
Thanks Jane,
I believe that the configuration "table.optimizer.source.predicate" has a
higher priority at the planner level than the configuration at the source level,
and it seems easy to implement now.
Best,
Jiabao
> 2023年10月24日 17:36,Jane Chan 写道:
>
> Hi Jiabao,
>
> Thanks for driving
Hi Jiabao,
I'm in favour of Jark's approach: while I can see the need for a
generic flag, I can also foresee the situation where users actually
want to be able to control it per connector. So why not go directly
for that approach?
Best regards,
Martijn
On Tue, Oct 24, 2023 at 11:37 AM Jane
Hi Jiabao,
Thanks for driving this discussion. I have a small question that will
"scan.filter-push-down.enabled" take precedence over
"table.optimizer.source.predicate" when the two parameters might conflict
each other?
Best,
Jane
On Tue, Oct 24, 2023 at 5:05 PM Jiabao Sun
wrote:
> Thanks
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 applyFilters method
to determine whether filter pushdown is needed.
This would increase complexity and
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
Thanks Leonard,
I have added sub-sections under the “Motivation" chapter to describe the
differences about them.
Best,
Jiabao
> 2023年10月24日 16:38,Leonard Xu 写道:
>
> Thanks @Jiabao for kicking off this discussion.
>
> Could you add a section to explain the difference between proposed
Thanks @Jiabao for kicking off this discussion.
Could you add a section to explain the difference between proposed connector
level config `scan.filter-push-down.enabled` and existing query level config
`table.optimizer.source.predicate-pushdown-enabled` ?
Best,
Leonard
> 2023年10月24日
Hi Devs,
I would like to start a discussion on FLIP-377: support configuration to
disable filter pushdown for Table/SQL Sources[1].
Currently, Flink Table/SQL does not expose fine-grained control for users to
enable or disable filter pushdown.
However, filter pushdown has some side effects,
69 matches
Mail list logo