[
https://issues.apache.org/jira/browse/FLINK-3849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15878666#comment-15878666
]
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_r102472465
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
---
@@ -53,13 +55,20 @@ class StreamTableSourceScan(
cluster,
traitSet,
getTable,
- tableSource
+ tableSource,
+ filterCondition
)
}
override def explainTerms(pw: RelWriter): RelWriter = {
- super.explainTerms(pw)
+ val terms = super.explainTerms(pw)
.item("fields",
TableEnvironment.getFieldNames(tableSource).mkString(", "))
+ if (filterCondition != null) {
+ import scala.collection.JavaConverters._
--- End diff --
Please move import up
> 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)