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

ASF GitHub Bot commented on FLINK-3849:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3520#discussion_r105726756
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSource.scala
 ---
    @@ -39,4 +39,6 @@ trait TableSource[T] {
       /** Returns the [[TypeInformation]] for the return type of the 
[[TableSource]]. */
       def getReturnType: TypeInformation[T]
     
    +  /** Describes the table source */
    +  def explainTerms(pw: RelWriter): RelWriter = pw
    --- End diff --
    
    Do we need this method? So far we tried to keep Calcite out of the 
TableSource interface. Can we change this to `explainSource(): String` and set 
the String in `BatchTableSourceScan` and `StreamTableSourceScan`?


> Add FilterableTableSource interface and translation rule
> --------------------------------------------------------
>
>                 Key: FLINK-3849
>                 URL: https://issues.apache.org/jira/browse/FLINK-3849
>             Project: Flink
>          Issue Type: New Feature
>          Components: Table API & SQL
>            Reporter: Fabian Hueske
>            Assignee: Kurt Young
>
> Add a {{FilterableTableSource}} interface for {{TableSource}} implementations 
> which support filter push-down.
> The interface could look as follows
> {code}
> def trait FilterableTableSource {
>   // returns unsupported predicate expression
>   def setPredicate(predicate: Expression): Expression
> }
> {code}
> In addition we need Calcite rules to push a predicate (or parts of it) into a 
> TableScan that refers to a {{FilterableTableSource}}. We might need to tweak 
> the cost model as well to push the optimizer in the right direction.



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

Reply via email to