You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "godfrey he (JIRA)" <ji...@apache.org> on 2017/02/24 06:00:49 UTC

[jira] [Comment Edited] (FLINK-5859) support partition pruning on Table API & SQL

    [ https://issues.apache.org/jira/browse/FLINK-5859?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15881999#comment-15881999 ] 

godfrey he edited comment on FLINK-5859 at 2/24/17 5:59 AM:
------------------------------------------------------------

Hi, [~fhueske], Thanks for your advice.
yes, partition pruning is a kind of coarse-grained filter push-down, both filter-pushdown and partition-pruning have common part that is extracting predicate from filter-condition base on the interest of different datasources. But, filter-pushdown and partition-pruning are independent concept in general. 
The following table shows that different datasources have different traits:

||Trait||Example||
|filter-pushdown only|MySQL, HBase|
|partiton-pruning only|CSV, TEXT|
|both filter-pushdown and partition-pruning| Parquet, Druid|

IMO, we should provide a clear concept as [~ykt836] mentioned above for developers, that includes both FilterableTableSource and PartitionableTableSource.

Looking forward to your advice, thanks.



was (Author: godfreyhe):
Hi, [~fhueske], Thanks for your advice.
yes, partition pruning is a kind of coarse-grained filter push-down, both filter-pushdown and partition-pruning have common parts that are extracting predicate from filter-condition base on the interest of different datasources. But, IMO, filter-pushdown and partition-pruning are independent concept in general. 
The following table shows that different datasources have different traits:

||Trait||Example||
|filter-pushdown only|MySQL, HBase|
|partiton-pruning only|CSV, TEXT|
|both filter-pushdown and partition-pruning| Parquet, Druid|

IMO, we should provide a clear concept as [~ykt836] mentioned above for developers, that includes both FilterableTableSource and PartitionableTableSource.

Looking forward to your advice, thanks.


> support partition pruning on Table API & SQL
> --------------------------------------------
>
>                 Key: FLINK-5859
>                 URL: https://issues.apache.org/jira/browse/FLINK-5859
>             Project: Flink
>          Issue Type: New Feature
>          Components: Table API & SQL
>            Reporter: godfrey he
>            Assignee: godfrey he
>
> Many data sources are partitionable storage, e.g. HDFS, Druid. And many queries just need to read a small subset of the total data. We can use partition information to prune or skip over files irrelevant to the user’s queries. Both query optimization time and execution time can be reduced obviously, especially for a large partitioned table.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)