[ https://issues.apache.org/jira/browse/FLINK-3849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15868731#comment-15868731 ]
ASF GitHub Bot commented on FLINK-3849: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101299952 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala --- @@ -58,8 +58,16 @@ class StreamTableSourceScan( } override def explainTerms(pw: RelWriter): RelWriter = { + val s = tableSource match { + case source: FilterableTableSource => + source.getPredicate.getOrElse("").toString.replaceAll("\\'|\\\"|\\s", "") + case _ => "" + } super.explainTerms(pw) .item("fields", TableEnvironment.getFieldNames(tableSource).mkString(", ")) + // TODO should we have this? If yes how it should look like, as in DataCalc? + // (current example, s = "id>2") --- End diff -- same as for `BatchTableSourceScan` > 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: Anton Solovev > > 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)