Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-3849

Add FilterableTableSource interface and translation rule

    Details

    • Type: New Feature
    • Status: Closed
    • Priority: Major
    • Resolution: Implemented
    • Affects Version/s: None
    • Fix Version/s: 1.3.0
    • Component/s: Table API & SQL
    • Labels:
      None

      Description

      Add a FilterableTableSource interface for TableSource implementations which support filter push-down.

      The interface could look as follows

      def trait FilterableTableSource {
        // returns unsupported predicate expression
        def setPredicate(predicate: Expression): Expression
      }
      

      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.

        Issue Links

          Activity

          Hide
          tonycox Anton Solovev added a comment -

          Hello, Fabian Hueske
          Can we implement it in CsvTableSource? What for is returning unsupported predicate expression needed?

          Show
          tonycox Anton Solovev added a comment - Hello, Fabian Hueske Can we implement it in CsvTableSource? What for is returning unsupported predicate expression needed?
          Hide
          fhueske Fabian Hueske added a comment -

          Hi Anton Solovev,

          I would like to wait for FLINK-3848 to be finalized before starting with this issue.
          Pushing filters into the CsvTableFormat does not provide a lot of benefits because we need to scan the whole file anyway.
          This feature is more suitable for querying a database (JDBC) or KV-store (HBase, Cassandra, ...) or reading data from a storage format such as Parquet or ORC which support filter pushdown.

          The unsupported expressions are necessary, because the FilterableTableSource needs to tell the optimizer which predicates it can apply and which predicates need to be evaluated by the query engine (i.e., Flink).

          Show
          fhueske Fabian Hueske added a comment - Hi Anton Solovev , I would like to wait for FLINK-3848 to be finalized before starting with this issue. Pushing filters into the CsvTableFormat does not provide a lot of benefits because we need to scan the whole file anyway. This feature is more suitable for querying a database (JDBC) or KV-store (HBase, Cassandra, ...) or reading data from a storage format such as Parquet or ORC which support filter pushdown. The unsupported expressions are necessary, because the FilterableTableSource needs to tell the optimizer which predicates it can apply and which predicates need to be evaluated by the query engine (i.e., Flink).
          Hide
          tonycox Anton Solovev added a comment -

          I think we need to parse an Expression from rexProgram.getCondition , and set the Expression to FilterableTableSource, then parse unsupported Expression into RexNode , right?

          Show
          tonycox Anton Solovev added a comment - I think we need to parse an Expression from rexProgram.getCondition , and set the Expression to FilterableTableSource , then parse unsupported Expression into RexNode , right?
          Hide
          fhueske Fabian Hueske added a comment -

          Yes, this would be the way to go, IMO. The translation from Expression to RexNode is already implemented in Expression.toRexNode().
          The translation of FilterableTableSource should follow the approach taken in FLINK-3848. In contrast to FLINK-3848, we do not need to take care of changing schema though.

          Show
          fhueske Fabian Hueske added a comment - Yes, this would be the way to go, IMO. The translation from Expression to RexNode is already implemented in Expression.toRexNode() . The translation of FilterableTableSource should follow the approach taken in FLINK-3848 . In contrast to FLINK-3848 , we do not need to take care of changing schema though.
          Hide
          tonycox Anton Solovev added a comment - - edited

          I think there is needed to create PushFilterProjectRule to compose these rules together

          Show
          tonycox Anton Solovev added a comment - - edited I think there is needed to create PushFilterProjectRule to compose these rules together
          Hide
          fhueske Fabian Hueske added a comment -

          Do you mean we need a single rule for pushing projection and filters into a BatchTableSourceScan, so basically extending the existing PushProjectIntoBatchTableSourceScanRule and PushProjectIntoStreamTableSourceScanRule?

          Can you explain why it would not be possible to have to separate rules?

          Thanks, Fabian

          Show
          fhueske Fabian Hueske added a comment - Do you mean we need a single rule for pushing projection and filters into a BatchTableSourceScan , so basically extending the existing PushProjectIntoBatchTableSourceScanRule and PushProjectIntoStreamTableSourceScanRule ? Can you explain why it would not be possible to have to separate rules? Thanks, Fabian
          Hide
          tonycox Anton Solovev added a comment -

          Yes, one rule for pushing them together. But, I`ve just realized your idea, I thought it wouldn`t match after one of the rule is applied

          Show
          tonycox Anton Solovev added a comment - Yes, one rule for pushing them together. But, I`ve just realized your idea, I thought it wouldn`t match after one of the rule is applied
          Hide
          tonycox Anton Solovev added a comment - - edited

          Hi Fabian Hueske, so setPredicate(predicate: Expression) returns unsupported expression. What if predicate looks like id > 2 || (amount * 2) > anydata and id > 2 part may be pushed into a source. Should we make sure that a source wouldn't do any logic mistakes before? And how,maybe add another method ensuring filtering or try to set predicate and a source make all checks.

          Show
          tonycox Anton Solovev added a comment - - edited Hi Fabian Hueske , so setPredicate(predicate: Expression) returns unsupported expression. What if predicate looks like id > 2 || (amount * 2) > anydata and id > 2 part may be pushed into a source. Should we make sure that a source wouldn't do any logic mistakes before? And how,maybe add another method ensuring filtering or try to set predicate and a source make all checks.
          Hide
          fhueske Fabian Hueske added a comment -

          The table source must make sure that the predicate is correctly evaluated.
          If the source cannot evaluate (amount * 2) > anydata, the predicate as a whole cannot be evaluated because any row might pass the filter regardless of id > 2 due to the disjunctive connection.

          So the source needs to know exactly what its capabilities are. If possible, it would be great if the table source could be called with predicates in CNF (conjunctive normal form).

          Show
          fhueske Fabian Hueske added a comment - The table source must make sure that the predicate is correctly evaluated. If the source cannot evaluate (amount * 2) > anydata , the predicate as a whole cannot be evaluated because any row might pass the filter regardless of id > 2 due to the disjunctive connection. So the source needs to know exactly what its capabilities are. If possible, it would be great if the table source could be called with predicates in CNF (conjunctive normal form).
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user tonycox opened a pull request:

          https://github.com/apache/flink/pull/3166

          FLINK-3849 [WIP] Add FilterableTableSource interface and Rules for pushing it

          Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
          If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
          In addition to going through the list, please provide a meaningful description of your changes.

          • [ ] General
          • The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
          • The pull request addresses only one issue
          • Each commit in the PR has a meaningful commit message (including the JIRA id)
          • [ ] Documentation
          • Documentation has been added for new functionality
          • Old documentation affected by the pull request has been updated
          • JavaDoc for public methods has been added
          • [ ] Tests & Build
          • Functionality added by the pull request is covered by tests
          • `mvn clean verify` has been executed successfully locally or a Travis build has passed

          I hove some questions:
          1) how to mock `TableScan` and `RelBuilder` in `#testRewriteRexProgramWithCondition`
          2) how to show filter predicate in `#explainTerms`

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/tonycox/flink filterableSource

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3166.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3166


          commit 73448a0425cb53f6b9db0bef5200537e2703a8b0
          Author: tonycox <anton_solovev@epam.com>
          Date: 2017-01-13T17:08:47Z

          FLINK-5481 Add type extraction from collection

          commit 0d7e86f6e37ded863f2b930e2d8ebba9a8fc1c07
          Author: tonycox <anton_solovev@epam.com>
          Date: 2017-01-17T15:31:15Z

          search types by columns

          commit 4f38c69c1a307d1910f8c4ad811fb32509248880
          Author: tonycox <anton_solovev@epam.com>
          Date: 2017-01-11T09:15:49Z

          FLINK-3849 Add FilterableTableSource interface and Rules for pushing it


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tonycox opened a pull request: https://github.com/apache/flink/pull/3166 FLINK-3849 [WIP] Add FilterableTableSource interface and Rules for pushing it Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration. If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide] ( http://flink.apache.org/how-to-contribute.html ). In addition to going through the list, please provide a meaningful description of your changes. [ ] General The pull request references the related JIRA issue (" [FLINK-XXX] Jira title text") The pull request addresses only one issue Each commit in the PR has a meaningful commit message (including the JIRA id) [ ] Documentation Documentation has been added for new functionality Old documentation affected by the pull request has been updated JavaDoc for public methods has been added [ ] Tests & Build Functionality added by the pull request is covered by tests `mvn clean verify` has been executed successfully locally or a Travis build has passed I hove some questions: 1) how to mock `TableScan` and `RelBuilder` in `#testRewriteRexProgramWithCondition` 2) how to show filter predicate in `#explainTerms` You can merge this pull request into a Git repository by running: $ git pull https://github.com/tonycox/flink filterableSource Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3166.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3166 commit 73448a0425cb53f6b9db0bef5200537e2703a8b0 Author: tonycox <anton_solovev@epam.com> Date: 2017-01-13T17:08:47Z FLINK-5481 Add type extraction from collection commit 0d7e86f6e37ded863f2b930e2d8ebba9a8fc1c07 Author: tonycox <anton_solovev@epam.com> Date: 2017-01-17T15:31:15Z search types by columns commit 4f38c69c1a307d1910f8c4ad811fb32509248880 Author: tonycox <anton_solovev@epam.com> Date: 2017-01-11T09:15:49Z FLINK-3849 Add FilterableTableSource interface and Rules for pushing it
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3166

          Thanks for working on this @tonycox. It seems that this PR contains commits that are not related to this issue (changes in TypeExtractor?). Can you remove those?

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3166 Thanks for working on this @tonycox. It seems that this PR contains commits that are not related to this issue (changes in TypeExtractor?). Can you remove those?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3166

          I quickly went through the code. IMHO, we should limit the expressions given to the FilterableTableSource to a minimum. E.g. if you look at the [HCatInputFormat](https://hive.apache.org/javadocs/hcat-r0.5.0/inputoutput.html#Filter+Operators) they only support very basic operations `'and', 'or', 'like', '()', '=', '<>' (not equal), '<', '>', '<=' and '>='`. I think that is also enough for our use case. We should also only support basic datatypes. Then we add a Filter after the TableSource in case the TableSource does not support every predicate. What do you think @fhueske?

          The expression translation should also be more robust, just parsing the Calcite string is very error-prone, can you construct the Expression using the case-classes for that?

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3166 I quickly went through the code. IMHO, we should limit the expressions given to the FilterableTableSource to a minimum. E.g. if you look at the [HCatInputFormat] ( https://hive.apache.org/javadocs/hcat-r0.5.0/inputoutput.html#Filter+Operators ) they only support very basic operations `'and', 'or', 'like', '()', '=', '<>' (not equal), '<', '>', '<=' and '>='`. I think that is also enough for our use case. We should also only support basic datatypes. Then we add a Filter after the TableSource in case the TableSource does not support every predicate. What do you think @fhueske? The expression translation should also be more robust, just parsing the Calcite string is very error-prone, can you construct the Expression using the case-classes for that?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

          https://github.com/apache/flink/pull/3166

          I'm not sure about limiting the operator types @twalthr. If the TableSource connects for example to a relational database, also complex predicates could be evaluated in the source. There might also be data stores that can handle more complex data types. In the end, the TableSource should decide which predicates to except or not.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3166 I'm not sure about limiting the operator types @twalthr. If the TableSource connects for example to a relational database, also complex predicates could be evaluated in the source. There might also be data stores that can handle more complex data types. In the end, the TableSource should decide which predicates to except or not.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3166

          @fhueske the problem is that we have currently no tool that converts RexNodes back to expressions. If we want to support all types of expressions, we need to implement a conversion method for all RexNodes (including Scalar functions) or send the RexNodes in the TableSource.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3166 @fhueske the problem is that we have currently no tool that converts RexNodes back to expressions. If we want to support all types of expressions, we need to implement a conversion method for all RexNodes (including Scalar functions) or send the RexNodes in the TableSource.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

          https://github.com/apache/flink/pull/3166

          Scalar functions can of course not be executed by the source.

          However, sources need to group the filter conditions into supported and unsupported expressions in any case. Unsupported would also include "I don't know this expression". So I think, there would not be additional implementation overhead on the TableSource side but potential gains in performance when pushing everything down.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3166 Scalar functions can of course not be executed by the source. However, sources need to group the filter conditions into supported and unsupported expressions in any case. Unsupported would also include "I don't know this expression". So I think, there would not be additional implementation overhead on the TableSource side but potential gains in performance when pushing everything down.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3166

          However I think we need a tool for converting RexNodes to Table API Expressions.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3166 However I think we need a tool for converting RexNodes to Table API Expressions.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3166

          I think it would be great if the mapping of Table API Expression to RexNode could happen within each Expression class. As the opposite of `toRexNode`. Instead of one large util class that does the conversion.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3166 I think it would be great if the mapping of Table API Expression to RexNode could happen within each Expression class. As the opposite of `toRexNode`. Instead of one large util class that does the conversion.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3166

          Hi @fhueske,
          what do you think about this PR?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3166 Hi @fhueske, what do you think about this PR?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101297805

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala —
          @@ -59,8 +59,16 @@ class BatchTableSourceScan(
          }

          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?
          — End diff –

          Yes, the filter should be in the explain string of the table source.
          I think it would be good if it was formatted as the filter in calc.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101297805 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala — @@ -59,8 +59,16 @@ class BatchTableSourceScan( } 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? — End diff – Yes, the filter should be in the explain string of the table source. I think it would be good if it was formatted as the filter in calc.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101394224

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,198 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.util
          +
          +import java.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlKind, SqlOperator}

          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.table.api.TableEnvironment
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.TableSource
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into expression
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + def extractExpression(rexProgram: RexProgram): Expression = {
          +
          + val refInputToName = getInputsWithNames(rexProgram)
          + val visitor = new ExpressionVisitor(refInputToName)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return null + }

          +
          + rexProgram.expandLocalRef(condition).accept(visitor)
          + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate)
          +
          + parsedExpression
          + }
          +
          + /**
          + * verify can the original expression be divided into `new` expression
          + * and remainder part without loss of logical correctness
          + *
          + * @param original initial expression
          + * @param lump part of original expression
          + * @return whether or not to decouple parts of the origin expression
          + */
          + def verifyExpressions(original: Expression, lump: Expression): Boolean = {
          + if (original == null & lump == null)

          { + return false + }
          + if (original.children.isEmpty | !checkOperator(original)) { + return false + }

          + val head = original.children.head
          + val last = original.children.last
          + if (head.checkEquals(lump))

          { + return checkOperator(original) + }
          + if (last.checkEquals(lump)) { + return checkOperator(original) + }

          + verifyExpressions(head, lump) match

          { + case true => true + case _ => verifyExpressions(last, lump) + }

          + }
          +
          + private def checkOperator(original: Expression): Boolean = {
          + original match

          { + case o: Or => false + case _ => true + }

          + }
          +
          + /**
          + * Generates a new RexProgram based on new expression.
          + *
          + * @param rexProgram original RexProgram
          + * @param scan input source
          + * @param expression filter condition (fields must be resolved)
          + * @param tableSource source to get names and type of table
          + * @param relBuilder builder for converting expression to Rex
          + */
          + def rewriteRexProgram(
          + rexProgram: RexProgram,
          + scan: TableScan,
          + expression: Expression,
          + tableSource: TableSource[_])(implicit relBuilder: RelBuilder): RexProgram = {
          — End diff –

          the `tableSource` parameter can be removed.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101394224 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.util + +import java.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlKind, SqlOperator} +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources.TableSource + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into expression + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + def extractExpression(rexProgram: RexProgram): Expression = { + + val refInputToName = getInputsWithNames(rexProgram) + val visitor = new ExpressionVisitor(refInputToName) + + val condition = rexProgram.getCondition + if (condition == null) { + return null + } + + rexProgram.expandLocalRef(condition).accept(visitor) + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate) + + parsedExpression + } + + /** + * verify can the original expression be divided into `new` expression + * and remainder part without loss of logical correctness + * + * @param original initial expression + * @param lump part of original expression + * @return whether or not to decouple parts of the origin expression + */ + def verifyExpressions(original: Expression, lump: Expression): Boolean = { + if (original == null & lump == null) { + return false + } + if (original.children.isEmpty | !checkOperator(original)) { + return false + } + val head = original.children.head + val last = original.children.last + if (head.checkEquals(lump)) { + return checkOperator(original) + } + if (last.checkEquals(lump)) { + return checkOperator(original) + } + verifyExpressions(head, lump) match { + case true => true + case _ => verifyExpressions(last, lump) + } + } + + private def checkOperator(original: Expression): Boolean = { + original match { + case o: Or => false + case _ => true + } + } + + /** + * Generates a new RexProgram based on new expression. + * + * @param rexProgram original RexProgram + * @param scan input source + * @param expression filter condition (fields must be resolved) + * @param tableSource source to get names and type of table + * @param relBuilder builder for converting expression to Rex + */ + def rewriteRexProgram( + rexProgram: RexProgram, + scan: TableScan, + expression: Expression, + tableSource: TableSource [_] )(implicit relBuilder: RelBuilder): RexProgram = { — End diff – the `tableSource` parameter can be removed.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101305739

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala —
          @@ -0,0 +1,72 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.datastream
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.datastream.

          {DataStreamCalc, StreamTableSourceScan}

          +import org.apache.flink.table.plan.rules.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule(
          — End diff –

          Same comments as for the `PushFilterIntoBatchTableSourceScanRule`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101305739 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala — @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.datastream. {DataStreamCalc, StreamTableSourceScan} +import org.apache.flink.table.plan.rules.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule( — End diff – Same comments as for the `PushFilterIntoBatchTableSourceScanRule`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101303197

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala —
          @@ -0,0 +1,72 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.dataSet
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.dataset.

          {BatchTableSourceScan, DataSetCalc}

          +import org.apache.flink.table.plan.rules.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataSetCalc],
          + operand(classOf[BatchTableSourceScan], none)),
          + "PushFilterIntoBatchTableSourceScanRule") {
          +
          + override def matches(call: RelOptRuleCall) = {
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          + scan.tableSource match

          { + case _: FilterableTableSource => true + case _ => false + }

          + }
          +
          + override def onMatch(call: RelOptRuleCall): Unit = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          +
          + val tableSource = scan.tableSource.asInstanceOf[FilterableTableSource]
          +
          + val expression = extractExpression(calc.calcProgram)
          + val unusedExpr = tableSource.setPredicate(expression)
          — End diff –

          Only call tableSource if predicate is not `None`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101303197 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala — @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.dataSet + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.dataset. {BatchTableSourceScan, DataSetCalc} +import org.apache.flink.table.plan.rules.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule( + operand(classOf [DataSetCalc] , + operand(classOf [BatchTableSourceScan] , none)), + "PushFilterIntoBatchTableSourceScanRule") { + + override def matches(call: RelOptRuleCall) = { + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource => true + case _ => false + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + + val tableSource = scan.tableSource.asInstanceOf [FilterableTableSource] + + val expression = extractExpression(calc.calcProgram) + val unusedExpr = tableSource.setPredicate(expression) — End diff – Only call tableSource if predicate is not `None`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101302395

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,198 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.util
          +
          +import java.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlKind, SqlOperator}

          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.table.api.TableEnvironment
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.TableSource
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into expression
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + def extractExpression(rexProgram: RexProgram): Expression = {
          — End diff –

          Rename to `extractPredicateExpression` and return `Option[Expression]`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101302395 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.util + +import java.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlKind, SqlOperator} +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources.TableSource + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into expression + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + def extractExpression(rexProgram: RexProgram): Expression = { — End diff – Rename to `extractPredicateExpression` and return `Option [Expression] `?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101394185

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,198 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.util
          +
          +import java.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlKind, SqlOperator}

          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.table.api.TableEnvironment
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.TableSource
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into expression
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + def extractExpression(rexProgram: RexProgram): Expression = {
          +
          + val refInputToName = getInputsWithNames(rexProgram)
          + val visitor = new ExpressionVisitor(refInputToName)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return null + }

          +
          + rexProgram.expandLocalRef(condition).accept(visitor)
          + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate)
          +
          + parsedExpression
          + }
          +
          + /**
          + * verify can the original expression be divided into `new` expression
          + * and remainder part without loss of logical correctness
          + *
          + * @param original initial expression
          + * @param lump part of original expression
          + * @return whether or not to decouple parts of the origin expression
          + */
          + def verifyExpressions(original: Expression, lump: Expression): Boolean = {
          + if (original == null & lump == null)

          { + return false + }
          + if (original.children.isEmpty | !checkOperator(original)) { + return false + }

          + val head = original.children.head
          + val last = original.children.last
          + if (head.checkEquals(lump))

          { + return checkOperator(original) + }
          + if (last.checkEquals(lump)) { + return checkOperator(original) + }

          + verifyExpressions(head, lump) match

          { + case true => true + case _ => verifyExpressions(last, lump) + }

          + }
          +
          + private def checkOperator(original: Expression): Boolean = {
          + original match

          { + case o: Or => false + case _ => true + }

          + }
          +
          + /**
          + * Generates a new RexProgram based on new expression.
          + *
          + * @param rexProgram original RexProgram
          + * @param scan input source
          + * @param expression filter condition (fields must be resolved)
          + * @param tableSource source to get names and type of table
          + * @param relBuilder builder for converting expression to Rex
          + */
          + def rewriteRexProgram(
          + rexProgram: RexProgram,
          + scan: TableScan,
          + expression: Expression,
          + tableSource: TableSource[_])(implicit relBuilder: RelBuilder): RexProgram = {
          +
          + if (expression != null) {
          +
          + val names = TableEnvironment.getFieldNames(tableSource)
          — End diff –

          We can get the name-type mapping also without the `tableSource` as follows:

          ```
          val inType = rexProgram.getInputRowType
          val fieldTypes: Map[String, TypeInformation[_]] = inType.getFieldList
          .map(f => f.getName -> FlinkTypeFactory.toTypeInfo(f.getType))
          .toMap
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101394185 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.util + +import java.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlKind, SqlOperator} +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources.TableSource + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into expression + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + def extractExpression(rexProgram: RexProgram): Expression = { + + val refInputToName = getInputsWithNames(rexProgram) + val visitor = new ExpressionVisitor(refInputToName) + + val condition = rexProgram.getCondition + if (condition == null) { + return null + } + + rexProgram.expandLocalRef(condition).accept(visitor) + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate) + + parsedExpression + } + + /** + * verify can the original expression be divided into `new` expression + * and remainder part without loss of logical correctness + * + * @param original initial expression + * @param lump part of original expression + * @return whether or not to decouple parts of the origin expression + */ + def verifyExpressions(original: Expression, lump: Expression): Boolean = { + if (original == null & lump == null) { + return false + } + if (original.children.isEmpty | !checkOperator(original)) { + return false + } + val head = original.children.head + val last = original.children.last + if (head.checkEquals(lump)) { + return checkOperator(original) + } + if (last.checkEquals(lump)) { + return checkOperator(original) + } + verifyExpressions(head, lump) match { + case true => true + case _ => verifyExpressions(last, lump) + } + } + + private def checkOperator(original: Expression): Boolean = { + original match { + case o: Or => false + case _ => true + } + } + + /** + * Generates a new RexProgram based on new expression. + * + * @param rexProgram original RexProgram + * @param scan input source + * @param expression filter condition (fields must be resolved) + * @param tableSource source to get names and type of table + * @param relBuilder builder for converting expression to Rex + */ + def rewriteRexProgram( + rexProgram: RexProgram, + scan: TableScan, + expression: Expression, + tableSource: TableSource [_] )(implicit relBuilder: RelBuilder): RexProgram = { + + if (expression != null) { + + val names = TableEnvironment.getFieldNames(tableSource) — End diff – We can get the name-type mapping also without the `tableSource` as follows: ``` val inType = rexProgram.getInputRowType val fieldTypes: Map[String, TypeInformation [_] ] = inType.getFieldList .map(f => f.getName -> FlinkTypeFactory.toTypeInfo(f.getType)) .toMap ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101397995

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/CommonTestData.scala —
          @@ -98,4 +106,134 @@ object CommonTestData

          { this(null, null) }

          }
          +
          + def getMockTableEnvironment: TableEnvironment = new MockTableEnvironment
          +
          + def getFilterableTableSource(
          + fieldNames: Array[String] = Array[String](
          + "name", "id", "amount", "price"),
          + fieldTypes: Array[TypeInformation[_]] = Array(
          + BasicTypeInfo.STRING_TYPE_INFO,
          + BasicTypeInfo.LONG_TYPE_INFO,
          + BasicTypeInfo.INT_TYPE_INFO,
          + BasicTypeInfo.DOUBLE_TYPE_INFO)) = new TestFilterableTableSource(fieldNames, fieldTypes)
          +}
          +
          +class MockTableEnvironment extends TableEnvironment(new TableConfig)

          { + + override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = ??? + + override protected def checkValidTableName(name: String): Unit = ??? + + override protected def getBuiltInRuleSet: RuleSet = ??? + + override def sql(query: String): Table = ??? + + override def registerTableSource(name: String, tableSource: TableSource[_]): Unit = ??? +}

          +
          +class TestFilterableTableSource(
          + fieldNames: Array[String],
          + fieldTypes: Array[TypeInformation[_]])
          + extends BatchTableSource[Row]
          + with StreamTableSource[Row]
          + with FilterableTableSource
          + with DefinedFieldNames {
          +
          + private var filterPredicate: Option[Expression] = None
          +
          + /** Returns the data of the table as a [[DataSet]]. */
          + override def getDataSet(execEnv: ExecutionEnvironment): DataSet[Row] =

          { + execEnv.fromCollection[Row]( + generateDynamicCollection(33, fieldNames, filterPredicate).asJava, getReturnType) + }
          +
          + /** Returns the data of the table as a [[DataStream]]. */
          + def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { + execEnv.fromCollection[Row]( + generateDynamicCollection(33, fieldNames, filterPredicate).asJava, getReturnType) + }

          +
          + private def generateDynamicCollection(
          + num: Int,
          + fieldNames: Array[String],
          + predicate: Option[Expression]): Seq[Row] = {
          +
          + if (predicate.isEmpty)

          { + throw new RuntimeException("filter expression was not set") + }

          +
          + val literal = predicate.get.children.last
          + .asInstanceOf[Literal]
          + .value.asInstanceOf[Int]
          +
          + def shouldCreateRow(value: Int): Boolean =

          { + value > literal + }

          +
          + def createRow(row: Row, name: String, pos: Int, value: Int): Unit = {
          — End diff –

          With hard-coded schema, this methods would not be necessary

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101397995 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/CommonTestData.scala — @@ -98,4 +106,134 @@ object CommonTestData { this(null, null) } } + + def getMockTableEnvironment: TableEnvironment = new MockTableEnvironment + + def getFilterableTableSource( + fieldNames: Array [String] = Array [String] ( + "name", "id", "amount", "price"), + fieldTypes: Array[TypeInformation [_] ] = Array( + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO)) = new TestFilterableTableSource(fieldNames, fieldTypes) +} + +class MockTableEnvironment extends TableEnvironment(new TableConfig) { + + override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = ??? + + override protected def checkValidTableName(name: String): Unit = ??? + + override protected def getBuiltInRuleSet: RuleSet = ??? + + override def sql(query: String): Table = ??? + + override def registerTableSource(name: String, tableSource: TableSource[_]): Unit = ??? +} + +class TestFilterableTableSource( + fieldNames: Array [String] , + fieldTypes: Array[TypeInformation [_] ]) + extends BatchTableSource [Row] + with StreamTableSource [Row] + with FilterableTableSource + with DefinedFieldNames { + + private var filterPredicate: Option [Expression] = None + + /** Returns the data of the table as a [ [DataSet] ]. */ + override def getDataSet(execEnv: ExecutionEnvironment): DataSet [Row] = { + execEnv.fromCollection[Row]( + generateDynamicCollection(33, fieldNames, filterPredicate).asJava, getReturnType) + } + + /** Returns the data of the table as a [ [DataStream] ]. */ + def getDataStream(execEnv: StreamExecutionEnvironment): DataStream [Row] = { + execEnv.fromCollection[Row]( + generateDynamicCollection(33, fieldNames, filterPredicate).asJava, getReturnType) + } + + private def generateDynamicCollection( + num: Int, + fieldNames: Array [String] , + predicate: Option [Expression] ): Seq [Row] = { + + if (predicate.isEmpty) { + throw new RuntimeException("filter expression was not set") + } + + val literal = predicate.get.children.last + .asInstanceOf [Literal] + .value.asInstanceOf [Int] + + def shouldCreateRow(value: Int): Boolean = { + value > literal + } + + def createRow(row: Row, name: String, pos: Int, value: Int): Unit = { — End diff – With hard-coded schema, this methods would not be necessary
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101316851

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,198 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.util
          +
          +import java.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlKind, SqlOperator}

          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.table.api.TableEnvironment
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.TableSource
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into expression
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + def extractExpression(rexProgram: RexProgram): Expression = {
          +
          + val refInputToName = getInputsWithNames(rexProgram)
          + val visitor = new ExpressionVisitor(refInputToName)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return null + }

          +
          + rexProgram.expandLocalRef(condition).accept(visitor)
          + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate)
          — End diff –

          Converting by generating and parsing strings is not very reliable.

          We should rather map `RexNode` directly to `Expression`.
          @twalthr suggested to add the translation logic to the corresponding `Expression` next to the `toRexNode()` method. We would need to find a "dictionary" to identify the relevant `Expression` for a `RexNode` though.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101316851 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.util + +import java.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlKind, SqlOperator} +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources.TableSource + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into expression + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + def extractExpression(rexProgram: RexProgram): Expression = { + + val refInputToName = getInputsWithNames(rexProgram) + val visitor = new ExpressionVisitor(refInputToName) + + val condition = rexProgram.getCondition + if (condition == null) { + return null + } + + rexProgram.expandLocalRef(condition).accept(visitor) + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate) — End diff – Converting by generating and parsing strings is not very reliable. We should rather map `RexNode` directly to `Expression`. @twalthr suggested to add the translation logic to the corresponding `Expression` next to the `toRexNode()` method. We would need to find a "dictionary" to identify the relevant `Expression` for a `RexNode` though.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101303533

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala —
          @@ -0,0 +1,72 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.dataSet
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.dataset.

          {BatchTableSourceScan, DataSetCalc}

          +import org.apache.flink.table.plan.rules.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataSetCalc],
          + operand(classOf[BatchTableSourceScan], none)),
          + "PushFilterIntoBatchTableSourceScanRule") {
          +
          + override def matches(call: RelOptRuleCall) = {
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          + scan.tableSource match

          { + case _: FilterableTableSource => true + case _ => false + }

          + }
          +
          + override def onMatch(call: RelOptRuleCall): Unit = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          +
          + val tableSource = scan.tableSource.asInstanceOf[FilterableTableSource]
          +
          + val expression = extractExpression(calc.calcProgram)
          + val unusedExpr = tableSource.setPredicate(expression)
          +
          + if (verifyExpressions(expression, unusedExpr)) {
          — End diff –

          Why do we need this check?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101303533 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala — @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.dataSet + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.dataset. {BatchTableSourceScan, DataSetCalc} +import org.apache.flink.table.plan.rules.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule( + operand(classOf [DataSetCalc] , + operand(classOf [BatchTableSourceScan] , none)), + "PushFilterIntoBatchTableSourceScanRule") { + + override def matches(call: RelOptRuleCall) = { + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource => true + case _ => false + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + + val tableSource = scan.tableSource.asInstanceOf [FilterableTableSource] + + val expression = extractExpression(calc.calcProgram) + val unusedExpr = tableSource.setPredicate(expression) + + if (verifyExpressions(expression, unusedExpr)) { — End diff – Why do we need this check?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101319389

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,198 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.util
          +
          +import java.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlKind, SqlOperator}

          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.table.api.TableEnvironment
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.TableSource
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into expression
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + def extractExpression(rexProgram: RexProgram): Expression = {
          +
          + val refInputToName = getInputsWithNames(rexProgram)
          + val visitor = new ExpressionVisitor(refInputToName)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return null + }

          +
          + rexProgram.expandLocalRef(condition).accept(visitor)
          — End diff –

          The condition should be converted into conjunctive normal form (CNF) before translating it. Calcite provides `RexUtil.toCnf()` to do that.

          The reason for CNF is that it makes the handling in `FilterableTableSource` much easier. We could also think about passing the conjunctive terms as a list to the `FilterableTableSource` and receiving a list of unsupported conjunctive terms. The list might be shorter but the elements may not change.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101319389 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.util + +import java.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlKind, SqlOperator} +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources.TableSource + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into expression + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + def extractExpression(rexProgram: RexProgram): Expression = { + + val refInputToName = getInputsWithNames(rexProgram) + val visitor = new ExpressionVisitor(refInputToName) + + val condition = rexProgram.getCondition + if (condition == null) { + return null + } + + rexProgram.expandLocalRef(condition).accept(visitor) — End diff – The condition should be converted into conjunctive normal form (CNF) before translating it. Calcite provides `RexUtil.toCnf()` to do that. The reason for CNF is that it makes the handling in `FilterableTableSource` much easier. We could also think about passing the conjunctive terms as a list to the `FilterableTableSource` and receiving a list of unsupported conjunctive terms. The list might be shorter but the elements may not change.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101310511

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,198 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.util
          +
          +import java.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlKind, SqlOperator}

          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.table.api.TableEnvironment
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.TableSource
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into expression
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + def extractExpression(rexProgram: RexProgram): Expression = {
          +
          + val refInputToName = getInputsWithNames(rexProgram)
          + val visitor = new ExpressionVisitor(refInputToName)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return null + }

          +
          + rexProgram.expandLocalRef(condition).accept(visitor)
          + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate)
          +
          + parsedExpression
          + }
          +
          + /**
          + * verify can the original expression be divided into `new` expression
          + * and remainder part without loss of logical correctness
          + *
          + * @param original initial expression
          + * @param lump part of original expression
          + * @return whether or not to decouple parts of the origin expression
          + */
          + def verifyExpressions(original: Expression, lump: Expression): Boolean = {
          + if (original == null & lump == null)

          { + return false + }
          + if (original.children.isEmpty | !checkOperator(original)) { + return false + }

          + val head = original.children.head
          + val last = original.children.last
          + if (head.checkEquals(lump))

          { + return checkOperator(original) + }
          + if (last.checkEquals(lump)) { + return checkOperator(original) + }

          + verifyExpressions(head, lump) match

          { + case true => true + case _ => verifyExpressions(last, lump) + }

          + }
          +
          + private def checkOperator(original: Expression): Boolean = {
          + original match

          { + case o: Or => false + case _ => true + }

          + }
          +
          + /**
          + * Generates a new RexProgram based on new expression.
          + *
          + * @param rexProgram original RexProgram
          + * @param scan input source
          + * @param expression filter condition (fields must be resolved)
          + * @param tableSource source to get names and type of table
          + * @param relBuilder builder for converting expression to Rex
          + */
          + def rewriteRexProgram(
          + rexProgram: RexProgram,
          + scan: TableScan,
          + expression: Expression,
          + tableSource: TableSource[_])(implicit relBuilder: RelBuilder): RexProgram = {
          +
          + if (expression != null) {
          +
          + val names = TableEnvironment.getFieldNames(tableSource)
          +
          + val nameToType = names
          + .zip(TableEnvironment.getFieldTypes(tableSource)).toMap
          +
          + relBuilder.push(scan)
          +
          + val rule: PartialFunction[Expression, Expression] =

          { + case u@UnresolvedFieldReference(name) => + ResolvedFieldReference(name, nameToType(name)) + }

          +
          + val newProjectExpressions = rewriteProjects(rexProgram, names)
          +
          + val resolvedExp = expression.postOrderTransform(rule)
          +
          + RexProgram.create(
          + rexProgram.getInputRowType,
          + newProjectExpressions,
          + resolvedExp.toRexNode,
          + rexProgram.getOutputRowType,
          + relBuilder.getRexBuilder)
          + } else

          { + rexProgram + }

          + }
          +
          + private def rewriteProjects(
          + rexProgram: RexProgram,
          + names: Array[String]): util.List[RexNode] =

          { + + val inputRewriter = new InputRewriter(names.indices.toArray) + val newProject = rexProgram.getProjectList.map( + exp => rexProgram.expandLocalRef(exp).accept(inputRewriter) + ).toList.asJava + newProject + }

          +
          + private def getInputsWithNames(rexProgram: RexProgram): Map[RexInputRef, String] = {
          + val names = rexProgram.getInputRowType.getFieldNames
          + rexProgram.getExprList.asScala.map

          { + case i: RexInputRef => + i -> names(i.getIndex) + case _ => null + }

          .filter(_ != null)
          — End diff –

          Filter on `RexInputRef` before performing the mapping?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101310511 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.util + +import java.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlKind, SqlOperator} +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources.TableSource + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into expression + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + def extractExpression(rexProgram: RexProgram): Expression = { + + val refInputToName = getInputsWithNames(rexProgram) + val visitor = new ExpressionVisitor(refInputToName) + + val condition = rexProgram.getCondition + if (condition == null) { + return null + } + + rexProgram.expandLocalRef(condition).accept(visitor) + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate) + + parsedExpression + } + + /** + * verify can the original expression be divided into `new` expression + * and remainder part without loss of logical correctness + * + * @param original initial expression + * @param lump part of original expression + * @return whether or not to decouple parts of the origin expression + */ + def verifyExpressions(original: Expression, lump: Expression): Boolean = { + if (original == null & lump == null) { + return false + } + if (original.children.isEmpty | !checkOperator(original)) { + return false + } + val head = original.children.head + val last = original.children.last + if (head.checkEquals(lump)) { + return checkOperator(original) + } + if (last.checkEquals(lump)) { + return checkOperator(original) + } + verifyExpressions(head, lump) match { + case true => true + case _ => verifyExpressions(last, lump) + } + } + + private def checkOperator(original: Expression): Boolean = { + original match { + case o: Or => false + case _ => true + } + } + + /** + * Generates a new RexProgram based on new expression. + * + * @param rexProgram original RexProgram + * @param scan input source + * @param expression filter condition (fields must be resolved) + * @param tableSource source to get names and type of table + * @param relBuilder builder for converting expression to Rex + */ + def rewriteRexProgram( + rexProgram: RexProgram, + scan: TableScan, + expression: Expression, + tableSource: TableSource [_] )(implicit relBuilder: RelBuilder): RexProgram = { + + if (expression != null) { + + val names = TableEnvironment.getFieldNames(tableSource) + + val nameToType = names + .zip(TableEnvironment.getFieldTypes(tableSource)).toMap + + relBuilder.push(scan) + + val rule: PartialFunction [Expression, Expression] = { + case u@UnresolvedFieldReference(name) => + ResolvedFieldReference(name, nameToType(name)) + } + + val newProjectExpressions = rewriteProjects(rexProgram, names) + + val resolvedExp = expression.postOrderTransform(rule) + + RexProgram.create( + rexProgram.getInputRowType, + newProjectExpressions, + resolvedExp.toRexNode, + rexProgram.getOutputRowType, + relBuilder.getRexBuilder) + } else { + rexProgram + } + } + + private def rewriteProjects( + rexProgram: RexProgram, + names: Array [String] ): util.List [RexNode] = { + + val inputRewriter = new InputRewriter(names.indices.toArray) + val newProject = rexProgram.getProjectList.map( + exp => rexProgram.expandLocalRef(exp).accept(inputRewriter) + ).toList.asJava + newProject + } + + private def getInputsWithNames(rexProgram: RexProgram): Map [RexInputRef, String] = { + val names = rexProgram.getInputRowType.getFieldNames + rexProgram.getExprList.asScala.map { + case i: RexInputRef => + i -> names(i.getIndex) + case _ => null + } .filter(_ != null) — End diff – Filter on `RexInputRef` before performing the mapping?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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`

          Show
          githubbot ASF GitHub Bot added a comment - 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`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101302191

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala —
          @@ -0,0 +1,72 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.dataSet
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.dataset.

          {BatchTableSourceScan, DataSetCalc}

          +import org.apache.flink.table.plan.rules.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataSetCalc],
          + operand(classOf[BatchTableSourceScan], none)),
          + "PushFilterIntoBatchTableSourceScanRule") {
          +
          + override def matches(call: RelOptRuleCall) = {
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          + scan.tableSource match

          { + case _: FilterableTableSource => true + case _ => false + }

          + }
          +
          + override def onMatch(call: RelOptRuleCall): Unit = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          +
          + val tableSource = scan.tableSource.asInstanceOf[FilterableTableSource]
          +
          + val expression = extractExpression(calc.calcProgram)
          + val unusedExpr = tableSource.setPredicate(expression)
          — End diff –

          rename to `remainingPredicate`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101302191 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala — @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.dataSet + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.dataset. {BatchTableSourceScan, DataSetCalc} +import org.apache.flink.table.plan.rules.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule( + operand(classOf [DataSetCalc] , + operand(classOf [BatchTableSourceScan] , none)), + "PushFilterIntoBatchTableSourceScanRule") { + + override def matches(call: RelOptRuleCall) = { + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource => true + case _ => false + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + + val tableSource = scan.tableSource.asInstanceOf [FilterableTableSource] + + val expression = extractExpression(calc.calcProgram) + val unusedExpr = tableSource.setPredicate(expression) — End diff – rename to `remainingPredicate`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101306528

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -0,0 +1,38 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.sources
          +
          +import org.apache.flink.table.expressions.Expression
          +
          +/**
          + * Adds support for filtering push-down to a [[TableSource]].
          + * A [[TableSource]] extending this interface is able to filter the fields of the return table.
          + *
          + */
          +trait FilterableTableSource {
          +
          + /** return an predicate expression that was set. */
          + def getPredicate: Option[Expression]
          — End diff –

          Do not use `Option` here. This interface might be implemented in Java.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101306528 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources + +import org.apache.flink.table.expressions.Expression + +/** + * Adds support for filtering push-down to a [ [TableSource] ]. + * A [ [TableSource] ] extending this interface is able to filter the fields of the return table. + * + */ +trait FilterableTableSource { + + /** return an predicate expression that was set. */ + def getPredicate: Option [Expression] — End diff – Do not use `Option` here. This interface might be implemented in Java.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r101393867

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,198 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.util
          +
          +import java.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlKind, SqlOperator}

          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.table.api.TableEnvironment
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.TableSource
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into expression
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + def extractExpression(rexProgram: RexProgram): Expression = {
          +
          + val refInputToName = getInputsWithNames(rexProgram)
          + val visitor = new ExpressionVisitor(refInputToName)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return null + }

          +
          + rexProgram.expandLocalRef(condition).accept(visitor)
          + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate)
          +
          + parsedExpression
          + }
          +
          + /**
          + * verify can the original expression be divided into `new` expression
          + * and remainder part without loss of logical correctness
          + *
          + * @param original initial expression
          + * @param lump part of original expression
          + * @return whether or not to decouple parts of the origin expression
          + */
          + def verifyExpressions(original: Expression, lump: Expression): Boolean = {
          + if (original == null & lump == null)

          { + return false + }
          + if (original.children.isEmpty | !checkOperator(original)) { + return false + }

          + val head = original.children.head
          + val last = original.children.last
          + if (head.checkEquals(lump))

          { + return checkOperator(original) + }
          + if (last.checkEquals(lump)) { + return checkOperator(original) + }

          + verifyExpressions(head, lump) match

          { + case true => true + case _ => verifyExpressions(last, lump) + }

          + }
          +
          + private def checkOperator(original: Expression): Boolean = {
          + original match

          { + case o: Or => false + case _ => true + }

          + }
          +
          + /**
          + * Generates a new RexProgram based on new expression.
          + *
          + * @param rexProgram original RexProgram
          + * @param scan input source
          + * @param expression filter condition (fields must be resolved)
          + * @param tableSource source to get names and type of table
          + * @param relBuilder builder for converting expression to Rex
          + */
          + def rewriteRexProgram(
          + rexProgram: RexProgram,
          + scan: TableScan,
          + expression: Expression,
          + tableSource: TableSource[_])(implicit relBuilder: RelBuilder): RexProgram = {
          +
          + if (expression != null) {
          +
          + val names = TableEnvironment.getFieldNames(tableSource)
          +
          + val nameToType = names
          + .zip(TableEnvironment.getFieldTypes(tableSource)).toMap
          +
          + relBuilder.push(scan)
          +
          + val rule: PartialFunction[Expression, Expression] =

          { + case u@UnresolvedFieldReference(name) => + ResolvedFieldReference(name, nameToType(name)) + }

          +
          + val newProjectExpressions = rewriteProjects(rexProgram, names)
          — End diff –

          `val projs = rexProgram.getProjectList.map(rexProgram.expandLocalRef)` should be sufficient here. Since we only apply a filter, the input schema did not change.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r101393867 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.util + +import java.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlKind, SqlOperator} +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources.TableSource + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into expression + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + def extractExpression(rexProgram: RexProgram): Expression = { + + val refInputToName = getInputsWithNames(rexProgram) + val visitor = new ExpressionVisitor(refInputToName) + + val condition = rexProgram.getCondition + if (condition == null) { + return null + } + + rexProgram.expandLocalRef(condition).accept(visitor) + val parsedExpression = ExpressionParser.parseExpression(visitor.getStringPredicate) + + parsedExpression + } + + /** + * verify can the original expression be divided into `new` expression + * and remainder part without loss of logical correctness + * + * @param original initial expression + * @param lump part of original expression + * @return whether or not to decouple parts of the origin expression + */ + def verifyExpressions(original: Expression, lump: Expression): Boolean = { + if (original == null & lump == null) { + return false + } + if (original.children.isEmpty | !checkOperator(original)) { + return false + } + val head = original.children.head + val last = original.children.last + if (head.checkEquals(lump)) { + return checkOperator(original) + } + if (last.checkEquals(lump)) { + return checkOperator(original) + } + verifyExpressions(head, lump) match { + case true => true + case _ => verifyExpressions(last, lump) + } + } + + private def checkOperator(original: Expression): Boolean = { + original match { + case o: Or => false + case _ => true + } + } + + /** + * Generates a new RexProgram based on new expression. + * + * @param rexProgram original RexProgram + * @param scan input source + * @param expression filter condition (fields must be resolved) + * @param tableSource source to get names and type of table + * @param relBuilder builder for converting expression to Rex + */ + def rewriteRexProgram( + rexProgram: RexProgram, + scan: TableScan, + expression: Expression, + tableSource: TableSource [_] )(implicit relBuilder: RelBuilder): RexProgram = { + + if (expression != null) { + + val names = TableEnvironment.getFieldNames(tableSource) + + val nameToType = names + .zip(TableEnvironment.getFieldTypes(tableSource)).toMap + + relBuilder.push(scan) + + val rule: PartialFunction [Expression, Expression] = { + case u@UnresolvedFieldReference(name) => + ResolvedFieldReference(name, nameToType(name)) + } + + val newProjectExpressions = rewriteProjects(rexProgram, names) — End diff – `val projs = rexProgram.getProjectList.map(rexProgram.expandLocalRef)` should be sufficient here. Since we only apply a filter, the input schema did not change.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3166

          @fhueske could you look at this approach of transfering `RexNode` to `Expression`, it's stil in wip state, but I need your adjustment

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3166 @fhueske could you look at this approach of transfering `RexNode` to `Expression`, it's stil in wip state, but I need your adjustment
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102473754

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,163 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.table.calcite.

          {FlinkTypeFactory, RexNodeWrapper}

          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.validate.FunctionCatalog
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.immutable.IndexedSeq
          +
          +object RexProgramExpressionExtractor {
          — End diff –

          Rename to `RexProgramPredicateExtractor`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102473754 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.calcite. {FlinkTypeFactory, RexNodeWrapper} +import org.apache.flink.table.expressions._ +import org.apache.flink.table.validate.FunctionCatalog + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.immutable.IndexedSeq + +object RexProgramExpressionExtractor { — End diff – Rename to `RexProgramPredicateExtractor`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102470146

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RexNodeWrapper.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.calcite
          +
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql._
          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.table.calcite.RexNodeWrapper._
          +
          +abstract class RexNodeWrapper(rex: RexNode) {
          — End diff –

          Please add some documentation about the purpose of this class.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102470146 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RexNodeWrapper.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.calcite + +import org.apache.calcite.rex._ +import org.apache.calcite.sql._ +import org.apache.flink.table.api.TableException +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.table.calcite.RexNodeWrapper._ + +abstract class RexNodeWrapper(rex: RexNode) { — End diff – Please add some documentation about the purpose of this class.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102507787

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableSourceTest.scala —
          @@ -126,21 +156,49 @@ class TableSourceTest extends TableTestBase {

          @Test
          def testStreamProjectableSourceScanNoIdentityCalc(): Unit =

          { - val (csvTable, tableName) = tableSource + val (tableSource, tableName) = csvTable val util = streamTestUtil() val tEnv = util.tEnv - tEnv.registerTableSource(tableName, csvTable) + tEnv.registerTableSource(tableName, tableSource) val result = tEnv .scan(tableName) .select('id, 'score, 'first) - val expected = sourceStreamTableNode(tableName, noCalcFields) + val expected = projectableSourceStreamTableNode(tableName, noCalcFields) util.verifyTable(result, expected) }

          @Test
          + def testStreamFilterableSourceScanPlanTableApi(): Unit = {
          + val (tableSource, tableName) = filterableTableSource
          + val util = streamTestUtil()
          + val tEnv = util.tEnv
          +
          + tEnv.registerTableSource(tableName, tableSource)
          +
          + val result = tEnv
          + .scan(tableName)
          + .select('price, 'id, 'amount)
          + .where("amount > 2 && price * 2 < 32")
          — End diff –

          An example for an unsupported predicate would be `'id.cast(BasicTypeInfo.STRING_TYPE_INFO) === "abc"`. This throws and exception when translating it to an `Expression`.

          As said before, unsupported expressions should be gracefully handled by not failing but instead by not offering this `RexNode` to the `FilterableTableSource` and evaluating it in the `DataSetCalc`.
          I would suggest to use `CAST` as an example to implement the graceful handling and adding support for it once the failure-free translation works.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102507787 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableSourceTest.scala — @@ -126,21 +156,49 @@ class TableSourceTest extends TableTestBase { @Test def testStreamProjectableSourceScanNoIdentityCalc(): Unit = { - val (csvTable, tableName) = tableSource + val (tableSource, tableName) = csvTable val util = streamTestUtil() val tEnv = util.tEnv - tEnv.registerTableSource(tableName, csvTable) + tEnv.registerTableSource(tableName, tableSource) val result = tEnv .scan(tableName) .select('id, 'score, 'first) - val expected = sourceStreamTableNode(tableName, noCalcFields) + val expected = projectableSourceStreamTableNode(tableName, noCalcFields) util.verifyTable(result, expected) } @Test + def testStreamFilterableSourceScanPlanTableApi(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = streamTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && price * 2 < 32") — End diff – An example for an unsupported predicate would be `'id.cast(BasicTypeInfo.STRING_TYPE_INFO) === "abc"`. This throws and exception when translating it to an `Expression`. As said before, unsupported expressions should be gracefully handled by not failing but instead by not offering this `RexNode` to the `FilterableTableSource` and evaluating it in the `DataSetCalc`. I would suggest to use `CAST` as an example to implement the graceful handling and adding support for it once the failure-free translation works.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102471926

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala —
          @@ -54,20 +55,27 @@ class BatchTableSourceScan(
          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 the import up

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102471926 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala — @@ -54,20 +55,27 @@ class BatchTableSourceScan( 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 the import up
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102476247

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RexNodeWrapper.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.calcite
          +
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql._
          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.table.calcite.RexNodeWrapper._
          +
          +abstract class RexNodeWrapper(rex: RexNode)

          { + def get: RexNode = rex + def toExpression(names: Map[RexInputRef, String]): Expression +}

          +
          +case class RexLiteralWrapper(literal: RexLiteral) extends RexNodeWrapper(literal) {
          + override def toExpression(names: Map[RexInputRef, String]): Expression =

          { + val typeInfo = FlinkTypeFactory.toTypeInfo(literal.getType) + Literal(literal.getValue, typeInfo) + }

          +}
          +
          +case class RexInputWrapper(input: RexInputRef) extends RexNodeWrapper(input) {
          + override def toExpression(names: Map[RexInputRef, String]): Expression =

          { + val typeInfo = FlinkTypeFactory.toTypeInfo(input.getType) + ResolvedFieldReference(names(input), typeInfo) + }

          +}
          +
          +case class RexCallWrapper(
          + call: RexCall,
          + operands: Seq[RexNodeWrapper]) extends RexNodeWrapper(call) {
          +
          + override def toExpression(names: Map[RexInputRef, String]): Expression = {
          + val ops = operands.map(_.toExpression(names))
          + call.op match {
          + case function: SqlFunction =>
          + lookupFunction(replace(function.getName), ops)
          + case postfix: SqlPostfixOperator =>
          + lookupFunction(replace(postfix.getName), ops)
          + case operator@_ =>
          + val name = replace(s"$

          {operator.kind}

          ")
          + lookupFunction(name, ops)
          + }
          + }
          +
          + def replace(str: String): String =

          { + str.replaceAll("\\s|_", "") + }

          +}
          +
          +object RexNodeWrapper {
          +
          + private var catalog: Option[FunctionCatalog] = None
          +
          + def wrap(rex: RexNode, functionCatalog: FunctionCatalog): RexNodeWrapper =

          { + catalog = Option(functionCatalog) + rex.accept(new WrapperVisitor) + }

          +
          + private[table] def lookupFunction(name: String, operands: Seq[Expression]): Expression =

          { + catalog.getOrElse(throw TableException("FunctionCatalog was not defined")) + .lookupFunction(name, operands) + }

          +}
          +
          +class WrapperVisitor extends RexVisitorImpl[RexNodeWrapper](true) {
          — End diff –

          We have to make sure that we do not miss anything here.
          IMO, we should try to translate as much as possible, but if something is not possible, we should make sure that we recognize that and do not offer this term to the `FilterableTableSource`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102476247 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RexNodeWrapper.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.calcite + +import org.apache.calcite.rex._ +import org.apache.calcite.sql._ +import org.apache.flink.table.api.TableException +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.table.calcite.RexNodeWrapper._ + +abstract class RexNodeWrapper(rex: RexNode) { + def get: RexNode = rex + def toExpression(names: Map[RexInputRef, String]): Expression +} + +case class RexLiteralWrapper(literal: RexLiteral) extends RexNodeWrapper(literal) { + override def toExpression(names: Map [RexInputRef, String] ): Expression = { + val typeInfo = FlinkTypeFactory.toTypeInfo(literal.getType) + Literal(literal.getValue, typeInfo) + } +} + +case class RexInputWrapper(input: RexInputRef) extends RexNodeWrapper(input) { + override def toExpression(names: Map [RexInputRef, String] ): Expression = { + val typeInfo = FlinkTypeFactory.toTypeInfo(input.getType) + ResolvedFieldReference(names(input), typeInfo) + } +} + +case class RexCallWrapper( + call: RexCall, + operands: Seq [RexNodeWrapper] ) extends RexNodeWrapper(call) { + + override def toExpression(names: Map [RexInputRef, String] ): Expression = { + val ops = operands.map(_.toExpression(names)) + call.op match { + case function: SqlFunction => + lookupFunction(replace(function.getName), ops) + case postfix: SqlPostfixOperator => + lookupFunction(replace(postfix.getName), ops) + case operator@_ => + val name = replace(s"$ {operator.kind} ") + lookupFunction(name, ops) + } + } + + def replace(str: String): String = { + str.replaceAll("\\s|_", "") + } +} + +object RexNodeWrapper { + + private var catalog: Option [FunctionCatalog] = None + + def wrap(rex: RexNode, functionCatalog: FunctionCatalog): RexNodeWrapper = { + catalog = Option(functionCatalog) + rex.accept(new WrapperVisitor) + } + + private [table] def lookupFunction(name: String, operands: Seq [Expression] ): Expression = { + catalog.getOrElse(throw TableException("FunctionCatalog was not defined")) + .lookupFunction(name, operands) + } +} + +class WrapperVisitor extends RexVisitorImpl [RexNodeWrapper] (true) { — End diff – We have to make sure that we do not miss anything here. IMO, we should try to translate as much as possible, but if something is not possible, we should make sure that we recognize that and do not offer this term to the `FilterableTableSource`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102485722

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala —
          @@ -0,0 +1,95 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.dataSet
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.dataset.

          {BatchTableSourceScan, DataSetCalc}

          +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataSetCalc],
          + operand(classOf[BatchTableSourceScan], none)),
          + "PushFilterIntoBatchTableSourceScanRule") {
          +
          + override def matches(call: RelOptRuleCall) = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          + scan.tableSource match

          { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + }

          + }
          +
          + override def onMatch(call: RelOptRuleCall): Unit = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          +
          + val filterableSource = scan.tableSource.asInstanceOf[FilterableTableSource]
          +
          + val program: RexProgram = calc.calcProgram
          + val tst = scan.getTable.unwrap(classOf[TableSourceTable[_]])
          + val predicate = extractPredicateExpressions(
          + program,
          + call.builder().getRexBuilder,
          + tst.tableEnv.getFunctionCatalog)
          +
          + if (predicate.length != 0) {
          + val remainingPredicate = filterableSource.setPredicate(predicate)
          +
          + if (verifyExpressions(predicate, remainingPredicate)) {
          +
          + val filterRexNode = getFilterExpressionAsRexNode(
          + program.getInputRowType,
          + scan,
          + predicate.diff(remainingPredicate))(call.builder())
          +
          + val newScan = new BatchTableSourceScan(
          + scan.getCluster,
          + scan.getTraitSet,
          + scan.getTable,
          + scan.tableSource,
          + filterRexNode)
          +
          + val newCalcProgram = rewriteRexProgram(
          — End diff –

          We would need to add those conjunctive terms which could not be translated by `extractPredicateExpressions()`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102485722 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala — @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.dataSet + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.dataset. {BatchTableSourceScan, DataSetCalc} +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule( + operand(classOf [DataSetCalc] , + operand(classOf [BatchTableSourceScan] , none)), + "PushFilterIntoBatchTableSourceScanRule") { + + override def matches(call: RelOptRuleCall) = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + + val filterableSource = scan.tableSource.asInstanceOf [FilterableTableSource] + + val program: RexProgram = calc.calcProgram + val tst = scan.getTable.unwrap(classOf[TableSourceTable [_] ]) + val predicate = extractPredicateExpressions( + program, + call.builder().getRexBuilder, + tst.tableEnv.getFunctionCatalog) + + if (predicate.length != 0) { + val remainingPredicate = filterableSource.setPredicate(predicate) + + if (verifyExpressions(predicate, remainingPredicate)) { + + val filterRexNode = getFilterExpressionAsRexNode( + program.getInputRowType, + scan, + predicate.diff(remainingPredicate))(call.builder()) + + val newScan = new BatchTableSourceScan( + scan.getCluster, + scan.getTraitSet, + scan.getTable, + scan.tableSource, + filterRexNode) + + val newCalcProgram = rewriteRexProgram( — End diff – We would need to add those conjunctive terms which could not be translated by `extractPredicateExpressions()`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102506293

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractorTest.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import java.math.BigDecimal
          +
          +import org.apache.calcite.adapter.java.JavaTypeFactory
          +import org.apache.calcite.plan._
          +import org.apache.calcite.plan.volcano.VolcanoPlanner
          +import org.apache.calcite.rel.`type`.

          {RelDataType, RelDataTypeSystem}

          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex.

          {RexBuilder, RexProgram, RexProgramBuilder}

          +import org.apache.calcite.sql.`type`.SqlTypeName._
          +import org.apache.calcite.sql.fun.SqlStdOperatorTable
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import org.apache.flink.table.expressions.

          {Expression, ExpressionParser}

          +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.plan.schema.CompositeRelDataType
          +import org.apache.flink.table.utils.CommonTestData
          +import org.junit.Test
          +import org.junit.Assert._
          +
          +import scala.collection.JavaConverters._
          +
          +class RexProgramExpressionExtractorTest {
          — End diff –

          I think it would be good to add a few more corner cases to the tests such as unsupported RexNodes or functions, single predicates, etc.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102506293 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractorTest.scala — @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import java.math.BigDecimal + +import org.apache.calcite.adapter.java.JavaTypeFactory +import org.apache.calcite.plan._ +import org.apache.calcite.plan.volcano.VolcanoPlanner +import org.apache.calcite.rel.`type`. {RelDataType, RelDataTypeSystem} +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex. {RexBuilder, RexProgram, RexProgramBuilder} +import org.apache.calcite.sql.`type`.SqlTypeName._ +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import org.apache.flink.table.expressions. {Expression, ExpressionParser} +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.plan.schema.CompositeRelDataType +import org.apache.flink.table.utils.CommonTestData +import org.junit.Test +import org.junit.Assert._ + +import scala.collection.JavaConverters._ + +class RexProgramExpressionExtractorTest { — End diff – I think it would be good to add a few more corner cases to the tests such as unsupported RexNodes or functions, single predicates, etc.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102491504

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,163 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.table.calcite.

          {FlinkTypeFactory, RexNodeWrapper}

          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.validate.FunctionCatalog
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.immutable.IndexedSeq
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into independent CNF expressions
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + private[flink] def extractPredicateExpressions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): Array[Expression] = {
          +
          + val fieldNames = getInputsWithNames(rexProgram)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return Array.empty + }

          + val call = rexProgram.expandLocalRef(condition)
          + val cnf = RexUtil.toCnf(rexBuilder, call)
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          + val expressions = conjunctions.asScala.map(
          + RexNodeWrapper.wrap(_, catalog).toExpression(fieldNames)
          + )
          + expressions.toArray
          + }
          +
          + /**
          + * verify should we apply remained expressions on
          + *
          + * @param original initial expression
          + * @param remained remained part of original expression
          + * @return whether or not to decouple parts of the origin expression
          + */
          + private[flink] def verifyExpressions(
          + original: Array[Expression],
          + remained: Array[Expression]): Boolean =
          + remained forall (original contains)
          +
          + /**
          + * Generates a new RexProgram based on new expression.
          + *
          + * @param rexProgram original RexProgram
          + * @param scan input source
          + * @param predicate filter condition (fields must be resolved)
          + * @param relBuilder builder for converting expression to Rex
          + */
          + private[flink] def rewriteRexProgram(
          + rexProgram: RexProgram,
          + scan: TableScan,
          + predicate: Array[Expression])(implicit relBuilder: RelBuilder): RexProgram =

          { + + relBuilder.push(scan) + + val inType = rexProgram.getInputRowType + val resolvedExps = resolveFields(predicate, inType) + val projs = rexProgram.getProjectList.map(rexProgram.expandLocalRef) + + RexProgram.create( + inType, + projs, + conjunct(resolvedExps).get.toRexNode, + rexProgram.getOutputRowType, + relBuilder.getRexBuilder) + }

          +
          + private[flink] def getFilterExpressionAsRexNode(
          + inputTpe: RelDataType,
          + scan: TableScan,
          + exps: Array[Expression])(implicit relBuilder: RelBuilder): RexNode = {
          + relBuilder.push(scan)
          + val resolvedExps = resolveFields(exps, inputTpe)
          + val fullExp = conjunct(resolvedExps)
          + if (fullExp.isDefined)

          { + fullExp.get.toRexNode + }

          else

          { + null + }

          + }
          +
          + private def resolveFields(
          + predicate: Array[Expression],
          + inType: RelDataType): Array[Expression] = {
          + val fieldTypes: Map[String, TypeInformation[_]] = inType.getFieldList
          + .map(f => f.getName -> FlinkTypeFactory.toTypeInfo(f.getType))
          + .toMap
          + val rule: PartialFunction[Expression, Expression] =

          { + case u@UnresolvedFieldReference(name) => + ResolvedFieldReference(name, fieldTypes(name)) + }

          + predicate.map(_.postOrderTransform(rule))
          + }
          +
          + private def conjunct(exps: Array[Expression]): Option[Expression] = {
          — End diff –

          This method could be more more concise if we use `reduce` instead of recursive pair-wise conjunctions, i.e.,

          `AND(a, AND(b, AND (c, d)))` instead of `AND(AND(a, b), AND(c, d))`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102491504 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.calcite. {FlinkTypeFactory, RexNodeWrapper} +import org.apache.flink.table.expressions._ +import org.apache.flink.table.validate.FunctionCatalog + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.immutable.IndexedSeq + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into independent CNF expressions + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + private [flink] def extractPredicateExpressions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): Array [Expression] = { + + val fieldNames = getInputsWithNames(rexProgram) + + val condition = rexProgram.getCondition + if (condition == null) { + return Array.empty + } + val call = rexProgram.expandLocalRef(condition) + val cnf = RexUtil.toCnf(rexBuilder, call) + val conjunctions = RelOptUtil.conjunctions(cnf) + val expressions = conjunctions.asScala.map( + RexNodeWrapper.wrap(_, catalog).toExpression(fieldNames) + ) + expressions.toArray + } + + /** + * verify should we apply remained expressions on + * + * @param original initial expression + * @param remained remained part of original expression + * @return whether or not to decouple parts of the origin expression + */ + private [flink] def verifyExpressions( + original: Array [Expression] , + remained: Array [Expression] ): Boolean = + remained forall (original contains) + + /** + * Generates a new RexProgram based on new expression. + * + * @param rexProgram original RexProgram + * @param scan input source + * @param predicate filter condition (fields must be resolved) + * @param relBuilder builder for converting expression to Rex + */ + private [flink] def rewriteRexProgram( + rexProgram: RexProgram, + scan: TableScan, + predicate: Array [Expression] )(implicit relBuilder: RelBuilder): RexProgram = { + + relBuilder.push(scan) + + val inType = rexProgram.getInputRowType + val resolvedExps = resolveFields(predicate, inType) + val projs = rexProgram.getProjectList.map(rexProgram.expandLocalRef) + + RexProgram.create( + inType, + projs, + conjunct(resolvedExps).get.toRexNode, + rexProgram.getOutputRowType, + relBuilder.getRexBuilder) + } + + private [flink] def getFilterExpressionAsRexNode( + inputTpe: RelDataType, + scan: TableScan, + exps: Array [Expression] )(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.push(scan) + val resolvedExps = resolveFields(exps, inputTpe) + val fullExp = conjunct(resolvedExps) + if (fullExp.isDefined) { + fullExp.get.toRexNode + } else { + null + } + } + + private def resolveFields( + predicate: Array [Expression] , + inType: RelDataType): Array [Expression] = { + val fieldTypes: Map[String, TypeInformation [_] ] = inType.getFieldList + .map(f => f.getName -> FlinkTypeFactory.toTypeInfo(f.getType)) + .toMap + val rule: PartialFunction [Expression, Expression] = { + case u@UnresolvedFieldReference(name) => + ResolvedFieldReference(name, fieldTypes(name)) + } + predicate.map(_.postOrderTransform(rule)) + } + + private def conjunct(exps: Array [Expression] ): Option [Expression] = { — End diff – This method could be more more concise if we use `reduce` instead of recursive pair-wise conjunctions, i.e., `AND(a, AND(b, AND (c, d)))` instead of `AND(AND(a, b), AND(c, d))`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102492031

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -0,0 +1,38 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.sources
          +
          +import org.apache.flink.table.expressions.Expression
          +
          +/**
          + * Adds support for filtering push-down to a [[TableSource]].
          + * A [[TableSource]] extending this interface is able to filter the fields of the return table.
          + *
          + */
          +trait FilterableTableSource {
          +
          + /** return an predicate expression that was set. */
          + def getPredicate: Array[Expression]
          +
          + /**
          + * @param predicate a filter expression that will be applied to fields to return.
          — End diff –

          The method docs should be more extensive. We have to explain that the expressions in the array are conjunctive terms which have to be accepted completely or not at all. All non-accepted terms have to be returned unmodified.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102492031 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources + +import org.apache.flink.table.expressions.Expression + +/** + * Adds support for filtering push-down to a [ [TableSource] ]. + * A [ [TableSource] ] extending this interface is able to filter the fields of the return table. + * + */ +trait FilterableTableSource { + + /** return an predicate expression that was set. */ + def getPredicate: Array [Expression] + + /** + * @param predicate a filter expression that will be applied to fields to return. — End diff – The method docs should be more extensive. We have to explain that the expressions in the array are conjunctive terms which have to be accepted completely or not at all. All non-accepted terms have to be returned unmodified.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102497345

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala —
          @@ -0,0 +1,95 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.datastream
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.datastream.

          {DataStreamCalc, StreamTableSourceScan}

          +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataStreamCalc],
          + operand(classOf[StreamTableSourceScan], none)),
          + "PushFilterIntoStreamTableSourceScanRule") {
          +
          + override def matches(call: RelOptRuleCall) = {
          + val calc: DataStreamCalc = call.rel(0).asInstanceOf[DataStreamCalc]
          + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
          + scan.tableSource match

          { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + }

          + }
          +
          + override def onMatch(call: RelOptRuleCall): Unit = {
          + val calc: DataStreamCalc = call.rel(0).asInstanceOf[DataStreamCalc]
          + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
          +
          + val filterableSource = scan.tableSource.asInstanceOf[FilterableTableSource]
          +
          + val program = calc.calcProgram
          + val tst = scan.getTable.unwrap(classOf[TableSourceTable[_]])
          + val predicates = extractPredicateExpressions(
          + program,
          + call.builder().getRexBuilder,
          + tst.tableEnv.getFunctionCatalog)
          +
          + if (predicates.length != 0) {
          + val remainingPredicate = filterableSource.setPredicate(predicates)
          — End diff –

          Do not continue if `remainingPredicate == predicates`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102497345 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala — @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.datastream. {DataStreamCalc, StreamTableSourceScan} +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule( + operand(classOf [DataStreamCalc] , + operand(classOf [StreamTableSourceScan] , none)), + "PushFilterIntoStreamTableSourceScanRule") { + + override def matches(call: RelOptRuleCall) = { + val calc: DataStreamCalc = call.rel(0).asInstanceOf [DataStreamCalc] + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf [StreamTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: DataStreamCalc = call.rel(0).asInstanceOf [DataStreamCalc] + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf [StreamTableSourceScan] + + val filterableSource = scan.tableSource.asInstanceOf [FilterableTableSource] + + val program = calc.calcProgram + val tst = scan.getTable.unwrap(classOf[TableSourceTable [_] ]) + val predicates = extractPredicateExpressions( + program, + call.builder().getRexBuilder, + tst.tableEnv.getFunctionCatalog) + + if (predicates.length != 0) { + val remainingPredicate = filterableSource.setPredicate(predicates) — End diff – Do not continue if `remainingPredicate == predicates`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102473702

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,163 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.table.calcite.

          {FlinkTypeFactory, RexNodeWrapper}

          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.validate.FunctionCatalog
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.immutable.IndexedSeq
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into independent CNF expressions
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + private[flink] def extractPredicateExpressions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): Array[Expression] = {
          +
          + val fieldNames = getInputsWithNames(rexProgram)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return Array.empty + }

          + val call = rexProgram.expandLocalRef(condition)
          — End diff –

          Please add a few comments

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102473702 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.calcite. {FlinkTypeFactory, RexNodeWrapper} +import org.apache.flink.table.expressions._ +import org.apache.flink.table.validate.FunctionCatalog + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.immutable.IndexedSeq + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into independent CNF expressions + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + private [flink] def extractPredicateExpressions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): Array [Expression] = { + + val fieldNames = getInputsWithNames(rexProgram) + + val condition = rexProgram.getCondition + if (condition == null) { + return Array.empty + } + val call = rexProgram.expandLocalRef(condition) — End diff – Please add a few comments
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102485241

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,163 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.table.calcite.

          {FlinkTypeFactory, RexNodeWrapper}

          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.validate.FunctionCatalog
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.immutable.IndexedSeq
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into independent CNF expressions
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + private[flink] def extractPredicateExpressions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): Array[Expression] = {
          — End diff –

          We should also return those `RexNodes` that we cannot translate, i.e., return (Array[Expression], Array[RexNode])

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102485241 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.calcite. {FlinkTypeFactory, RexNodeWrapper} +import org.apache.flink.table.expressions._ +import org.apache.flink.table.validate.FunctionCatalog + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.immutable.IndexedSeq + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into independent CNF expressions + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + private [flink] def extractPredicateExpressions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): Array [Expression] = { — End diff – We should also return those `RexNodes` that we cannot translate, i.e., return (Array [Expression] , Array [RexNode] )
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102486920

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala —
          @@ -0,0 +1,95 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.dataSet
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.dataset.

          {BatchTableSourceScan, DataSetCalc}

          +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataSetCalc],
          + operand(classOf[BatchTableSourceScan], none)),
          + "PushFilterIntoBatchTableSourceScanRule") {
          +
          + override def matches(call: RelOptRuleCall) = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          + scan.tableSource match

          { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + }

          + }
          +
          + override def onMatch(call: RelOptRuleCall): Unit = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          +
          + val filterableSource = scan.tableSource.asInstanceOf[FilterableTableSource]
          +
          + val program: RexProgram = calc.calcProgram
          + val tst = scan.getTable.unwrap(classOf[TableSourceTable[_]])
          + val predicate = extractPredicateExpressions(
          + program,
          + call.builder().getRexBuilder,
          + tst.tableEnv.getFunctionCatalog)
          +
          + if (predicate.length != 0) {
          + val remainingPredicate = filterableSource.setPredicate(predicate)
          +
          + if (verifyExpressions(predicate, remainingPredicate)) {
          — End diff –

          The `FilterableTableSource` violates the method contract if this is not true. I would log a WARN message here.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102486920 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala — @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.dataSet + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.dataset. {BatchTableSourceScan, DataSetCalc} +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule( + operand(classOf [DataSetCalc] , + operand(classOf [BatchTableSourceScan] , none)), + "PushFilterIntoBatchTableSourceScanRule") { + + override def matches(call: RelOptRuleCall) = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + + val filterableSource = scan.tableSource.asInstanceOf [FilterableTableSource] + + val program: RexProgram = calc.calcProgram + val tst = scan.getTable.unwrap(classOf[TableSourceTable [_] ]) + val predicate = extractPredicateExpressions( + program, + call.builder().getRexBuilder, + tst.tableEnv.getFunctionCatalog) + + if (predicate.length != 0) { + val remainingPredicate = filterableSource.setPredicate(predicate) + + if (verifyExpressions(predicate, remainingPredicate)) { — End diff – The `FilterableTableSource` violates the method contract if this is not true. I would log a WARN message here.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102487221

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,163 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.table.calcite.

          {FlinkTypeFactory, RexNodeWrapper}

          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.validate.FunctionCatalog
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.immutable.IndexedSeq
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into independent CNF expressions
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + private[flink] def extractPredicateExpressions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): Array[Expression] = {
          +
          + val fieldNames = getInputsWithNames(rexProgram)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return Array.empty + }

          + val call = rexProgram.expandLocalRef(condition)
          + val cnf = RexUtil.toCnf(rexBuilder, call)
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          + val expressions = conjunctions.asScala.map(
          + RexNodeWrapper.wrap(_, catalog).toExpression(fieldNames)
          + )
          + expressions.toArray
          + }
          +
          + /**
          + * verify should we apply remained expressions on
          + *
          + * @param original initial expression
          + * @param remained remained part of original expression
          + * @return whether or not to decouple parts of the origin expression
          + */
          + private[flink] def verifyExpressions(
          + original: Array[Expression],
          + remained: Array[Expression]): Boolean =
          + remained forall (original contains)
          +
          + /**
          + * Generates a new RexProgram based on new expression.
          + *
          + * @param rexProgram original RexProgram
          + * @param scan input source
          + * @param predicate filter condition (fields must be resolved)
          + * @param relBuilder builder for converting expression to Rex
          + */
          + private[flink] def rewriteRexProgram(
          + rexProgram: RexProgram,
          + scan: TableScan,
          + predicate: Array[Expression])(implicit relBuilder: RelBuilder): RexProgram = {
          — End diff –

          We need to inject all conjunctive `RexNode` terms which could not be translated into `Expression` here as well.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102487221 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.calcite. {FlinkTypeFactory, RexNodeWrapper} +import org.apache.flink.table.expressions._ +import org.apache.flink.table.validate.FunctionCatalog + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.immutable.IndexedSeq + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into independent CNF expressions + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + private [flink] def extractPredicateExpressions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): Array [Expression] = { + + val fieldNames = getInputsWithNames(rexProgram) + + val condition = rexProgram.getCondition + if (condition == null) { + return Array.empty + } + val call = rexProgram.expandLocalRef(condition) + val cnf = RexUtil.toCnf(rexBuilder, call) + val conjunctions = RelOptUtil.conjunctions(cnf) + val expressions = conjunctions.asScala.map( + RexNodeWrapper.wrap(_, catalog).toExpression(fieldNames) + ) + expressions.toArray + } + + /** + * verify should we apply remained expressions on + * + * @param original initial expression + * @param remained remained part of original expression + * @return whether or not to decouple parts of the origin expression + */ + private [flink] def verifyExpressions( + original: Array [Expression] , + remained: Array [Expression] ): Boolean = + remained forall (original contains) + + /** + * Generates a new RexProgram based on new expression. + * + * @param rexProgram original RexProgram + * @param scan input source + * @param predicate filter condition (fields must be resolved) + * @param relBuilder builder for converting expression to Rex + */ + private [flink] def rewriteRexProgram( + rexProgram: RexProgram, + scan: TableScan, + predicate: Array [Expression] )(implicit relBuilder: RelBuilder): RexProgram = { — End diff – We need to inject all conjunctive `RexNode` terms which could not be translated into `Expression` here as well.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102484272

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala —
          @@ -0,0 +1,163 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.rel.core.TableScan
          +import org.apache.calcite.rex._
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.tools.RelBuilder
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.table.calcite.

          {FlinkTypeFactory, RexNodeWrapper}

          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.validate.FunctionCatalog
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.immutable.IndexedSeq
          +
          +object RexProgramExpressionExtractor {
          +
          + /**
          + * converts a rexProgram condition into independent CNF expressions
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expression
          + */
          + private[flink] def extractPredicateExpressions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): Array[Expression] = {
          +
          + val fieldNames = getInputsWithNames(rexProgram)
          +
          + val condition = rexProgram.getCondition
          + if (condition == null)

          { + return Array.empty + }

          + val call = rexProgram.expandLocalRef(condition)
          + val cnf = RexUtil.toCnf(rexBuilder, call)
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          + val expressions = conjunctions.asScala.map(
          + RexNodeWrapper.wrap(_, catalog).toExpression(fieldNames)
          + )
          + expressions.toArray
          + }
          +
          + /**
          + * verify should we apply remained expressions on
          — End diff –

          Complete comment

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102484272 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExpressionExtractor.scala — @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rex._ +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.calcite. {FlinkTypeFactory, RexNodeWrapper} +import org.apache.flink.table.expressions._ +import org.apache.flink.table.validate.FunctionCatalog + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.immutable.IndexedSeq + +object RexProgramExpressionExtractor { + + /** + * converts a rexProgram condition into independent CNF expressions + * + * @param rexProgram The RexProgram to analyze + * @return converted expression + */ + private [flink] def extractPredicateExpressions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): Array [Expression] = { + + val fieldNames = getInputsWithNames(rexProgram) + + val condition = rexProgram.getCondition + if (condition == null) { + return Array.empty + } + val call = rexProgram.expandLocalRef(condition) + val cnf = RexUtil.toCnf(rexBuilder, call) + val conjunctions = RelOptUtil.conjunctions(cnf) + val expressions = conjunctions.asScala.map( + RexNodeWrapper.wrap(_, catalog).toExpression(fieldNames) + ) + expressions.toArray + } + + /** + * verify should we apply remained expressions on — End diff – Complete comment
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102886003

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala —
          @@ -0,0 +1,95 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.datastream
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.datastream.

          {DataStreamCalc, StreamTableSourceScan}

          +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataStreamCalc],
          + operand(classOf[StreamTableSourceScan], none)),
          + "PushFilterIntoStreamTableSourceScanRule") {
          +
          + override def matches(call: RelOptRuleCall) = {
          + val calc: DataStreamCalc = call.rel(0).asInstanceOf[DataStreamCalc]
          + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
          + scan.tableSource match

          { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + }

          + }
          +
          + override def onMatch(call: RelOptRuleCall): Unit = {
          + val calc: DataStreamCalc = call.rel(0).asInstanceOf[DataStreamCalc]
          + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
          +
          + val filterableSource = scan.tableSource.asInstanceOf[FilterableTableSource]
          +
          + val program = calc.calcProgram
          + val tst = scan.getTable.unwrap(classOf[TableSourceTable[_]])
          + val predicates = extractPredicateExpressions(
          + program,
          + call.builder().getRexBuilder,
          + tst.tableEnv.getFunctionCatalog)
          +
          + if (predicates.length != 0) {
          + val remainingPredicate = filterableSource.setPredicate(predicates)
          — End diff –

          if remainingPredicate is empty, we should remove calc node also.

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102886003 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala — @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.datastream. {DataStreamCalc, StreamTableSourceScan} +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule( + operand(classOf [DataStreamCalc] , + operand(classOf [StreamTableSourceScan] , none)), + "PushFilterIntoStreamTableSourceScanRule") { + + override def matches(call: RelOptRuleCall) = { + val calc: DataStreamCalc = call.rel(0).asInstanceOf [DataStreamCalc] + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf [StreamTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: DataStreamCalc = call.rel(0).asInstanceOf [DataStreamCalc] + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf [StreamTableSourceScan] + + val filterableSource = scan.tableSource.asInstanceOf [FilterableTableSource] + + val program = calc.calcProgram + val tst = scan.getTable.unwrap(classOf[TableSourceTable [_] ]) + val predicates = extractPredicateExpressions( + program, + call.builder().getRexBuilder, + tst.tableEnv.getFunctionCatalog) + + if (predicates.length != 0) { + val remainingPredicate = filterableSource.setPredicate(predicates) — End diff – if remainingPredicate is empty, we should remove calc node also.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3166#discussion_r102912230

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala —
          @@ -0,0 +1,95 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.datastream
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.datastream.

          {DataStreamCalc, StreamTableSourceScan}

          +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataStreamCalc],
          + operand(classOf[StreamTableSourceScan], none)),
          + "PushFilterIntoStreamTableSourceScanRule") {
          +
          + override def matches(call: RelOptRuleCall) = {
          + val calc: DataStreamCalc = call.rel(0).asInstanceOf[DataStreamCalc]
          + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
          + scan.tableSource match

          { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + }

          + }
          +
          + override def onMatch(call: RelOptRuleCall): Unit = {
          + val calc: DataStreamCalc = call.rel(0).asInstanceOf[DataStreamCalc]
          + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
          +
          + val filterableSource = scan.tableSource.asInstanceOf[FilterableTableSource]
          +
          + val program = calc.calcProgram
          + val tst = scan.getTable.unwrap(classOf[TableSourceTable[_]])
          + val predicates = extractPredicateExpressions(
          + program,
          + call.builder().getRexBuilder,
          + tst.tableEnv.getFunctionCatalog)
          +
          + if (predicates.length != 0) {
          + val remainingPredicate = filterableSource.setPredicate(predicates)
          — End diff –

          Right, but only if it does not do any projection.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3166#discussion_r102912230 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala — @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.datastream. {DataStreamCalc, StreamTableSourceScan} +import org.apache.flink.table.plan.util.RexProgramExpressionExtractor._ +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule( + operand(classOf [DataStreamCalc] , + operand(classOf [StreamTableSourceScan] , none)), + "PushFilterIntoStreamTableSourceScanRule") { + + override def matches(call: RelOptRuleCall) = { + val calc: DataStreamCalc = call.rel(0).asInstanceOf [DataStreamCalc] + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf [StreamTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource => + calc.calcProgram.getCondition != null + case _ => false + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: DataStreamCalc = call.rel(0).asInstanceOf [DataStreamCalc] + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf [StreamTableSourceScan] + + val filterableSource = scan.tableSource.asInstanceOf [FilterableTableSource] + + val program = calc.calcProgram + val tst = scan.getTable.unwrap(classOf[TableSourceTable [_] ]) + val predicates = extractPredicateExpressions( + program, + call.builder().getRexBuilder, + tst.tableEnv.getFunctionCatalog) + + if (predicates.length != 0) { + val remainingPredicate = filterableSource.setPredicate(predicates) — End diff – Right, but only if it does not do any projection.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

          https://github.com/apache/flink/pull/3166

          Hi @tonycox, what are your plans regarding this PR?
          I'm asking because we would like to put another feature (FLINK-5859(https://issues.apache.org/jira/browse/FLINK-5859): Partition Pruning TableSource) on top of `FilterableTableSource`.

          Thanks, Fabian

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3166 Hi @tonycox, what are your plans regarding this PR? I'm asking because we would like to put another feature ( FLINK-5859 ( https://issues.apache.org/jira/browse/FLINK-5859): Partition Pruning TableSource) on top of `FilterableTableSource`. Thanks, Fabian
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3166

          Hi @fhueske I cant continue on this PR, have not enough time for now. If you need implementation of it immediately I will unassign

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3166 Hi @fhueske I cant continue on this PR, have not enough time for now. If you need implementation of it immediately I will unassign
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

          https://github.com/apache/flink/pull/3166

          OK, thanks for the quick response!
          I'll point the contributor of the partition pruning to this PR.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3166 OK, thanks for the quick response! I'll point the contributor of the partition pruning to this PR.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user KurtYoung opened a pull request:

          https://github.com/apache/flink/pull/3520

          FLINK-3849 [table] Add FilterableTableSource interface and rules for pushing it

          This PR is based on #3166 , and added following changes:

          1. Refactor `RexProgramExpressionExtractor` and `RexProgramExpressionExtractor` to `RexProgramExtractor` and `RexProgramRewriter`. `RexProgramExtractor` is responsible for retract either projection expressions or filter expression.
          2. Make sure we don't fail during extracting and converting filter RexNodes to expressions. The expressions which successfully translated and unconverted RexNodes will both be returned.
          3. Add some tests for `RexProgramExtractor`.
          4. Provide unified `PushFilterIntoTableSourceScanRuleBase` to support filter push down in both batch and stream mode.
          5. Add some logical tests for filter push down in different situations, like fully push down and partial push down.
          5. Slight change of testing class `TestFilterableTableSource` to make it less specialized.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/KurtYoung/flink flink-3849

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3520.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3520


          commit 0a7af41509d9a0db3e791cb9f4dc5a1a8086f0b2
          Author: tonycox <anton_solovev@epam.com>
          Date: 2017-01-11T09:15:49Z

          FLINK-3849 Add FilterableTableSource interface and Rules for pushing it

          commit 549b4e00e68d32f070e196fc6eb9a7f5f9e937c3
          Author: tonycox <anton_solovev@epam.com>
          Date: 2017-01-31T12:41:52Z

          fix filterable test

          commit 9aa82062832e0aabcb003e582c8130aeecc91a73
          Author: tonycox <anton_solovev@epam.com>
          Date: 2017-02-16T13:32:33Z

          rebase and trying fix rexnode parsing

          commit 646a6931224c7dcc58501ec014ab675925bb105d
          Author: tonycox <anton_solovev@epam.com>
          Date: 2017-02-17T16:48:40Z

          create wrapper and update rules

          commit abfa38d894aa86b7a5c91dd29bf398b880c8bfe7
          Author: Kurt Young <ykt836@gmail.com>
          Date: 2017-03-13T07:30:13Z

          FLINK-3849 [table] Add FilterableTableSource interface and rules for pushing it


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user KurtYoung opened a pull request: https://github.com/apache/flink/pull/3520 FLINK-3849 [table] Add FilterableTableSource interface and rules for pushing it This PR is based on #3166 , and added following changes: 1. Refactor `RexProgramExpressionExtractor` and `RexProgramExpressionExtractor` to `RexProgramExtractor` and `RexProgramRewriter`. `RexProgramExtractor` is responsible for retract either projection expressions or filter expression. 2. Make sure we don't fail during extracting and converting filter RexNodes to expressions. The expressions which successfully translated and unconverted RexNodes will both be returned. 3. Add some tests for `RexProgramExtractor`. 4. Provide unified `PushFilterIntoTableSourceScanRuleBase` to support filter push down in both batch and stream mode. 5. Add some logical tests for filter push down in different situations, like fully push down and partial push down. 5. Slight change of testing class `TestFilterableTableSource` to make it less specialized. You can merge this pull request into a Git repository by running: $ git pull https://github.com/KurtYoung/flink flink-3849 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3520.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3520 commit 0a7af41509d9a0db3e791cb9f4dc5a1a8086f0b2 Author: tonycox <anton_solovev@epam.com> Date: 2017-01-11T09:15:49Z FLINK-3849 Add FilterableTableSource interface and Rules for pushing it commit 549b4e00e68d32f070e196fc6eb9a7f5f9e937c3 Author: tonycox <anton_solovev@epam.com> Date: 2017-01-31T12:41:52Z fix filterable test commit 9aa82062832e0aabcb003e582c8130aeecc91a73 Author: tonycox <anton_solovev@epam.com> Date: 2017-02-16T13:32:33Z rebase and trying fix rexnode parsing commit 646a6931224c7dcc58501ec014ab675925bb105d Author: tonycox <anton_solovev@epam.com> Date: 2017-02-17T16:48:40Z create wrapper and update rules commit abfa38d894aa86b7a5c91dd29bf398b880c8bfe7 Author: Kurt Young <ykt836@gmail.com> Date: 2017-03-13T07:30:13Z FLINK-3849 [table] Add FilterableTableSource interface and rules for pushing it
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r105811515

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala —
          @@ -42,63 +41,40 @@ class DataSetCalc(
          traitSet: RelTraitSet,
          input: RelNode,
          rowRelDataType: RelDataType,

          • private[flink] val calcProgram: RexProgram, // for tests
            + calcProgram: RexProgram,
            ruleDescription: String)
          • extends SingleRel(cluster, traitSet, input)
            + extends Calc(cluster, traitSet, input, calcProgram)
            with CommonCalc
            with DataSetRel {
          • override def deriveRowType() = rowRelDataType
            + override def deriveRowType(): RelDataType = rowRelDataType
          • override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
          • new DataSetCalc(
          • cluster,
          • traitSet,
          • inputs.get(0),
          • getRowType,
          • calcProgram,
          • ruleDescription)
            + override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { + new DataSetCalc(cluster, traitSet, child, getRowType, program, ruleDescription) }

          override def toString: String = calcToString(calcProgram, getExpressionString)

          override def explainTerms(pw: RelWriter): RelWriter = {

          • super.explainTerms(pw)
          • .item("select", selectionToString(calcProgram, getExpressionString))
          • .itemIf("where",
          • conditionToString(calcProgram, getExpressionString),
          • calcProgram.getCondition != null)
            + pw.input("input", getInput)
            + .item("select", selectionToString(calcProgram, getExpressionString))
              • End diff –

          two spaces for indention

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r105811515 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala — @@ -42,63 +41,40 @@ class DataSetCalc( traitSet: RelTraitSet, input: RelNode, rowRelDataType: RelDataType, private [flink] val calcProgram: RexProgram, // for tests + calcProgram: RexProgram, ruleDescription: String) extends SingleRel(cluster, traitSet, input) + extends Calc(cluster, traitSet, input, calcProgram) with CommonCalc with DataSetRel { override def deriveRowType() = rowRelDataType + override def deriveRowType(): RelDataType = rowRelDataType override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { new DataSetCalc( cluster, traitSet, inputs.get(0), getRowType, calcProgram, ruleDescription) + override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { + new DataSetCalc(cluster, traitSet, child, getRowType, program, ruleDescription) } override def toString: String = calcToString(calcProgram, getExpressionString) override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) .item("select", selectionToString(calcProgram, getExpressionString)) .itemIf("where", conditionToString(calcProgram, getExpressionString), calcProgram.getCondition != null) + pw.input("input", getInput) + .item("select", selectionToString(calcProgram, getExpressionString)) End diff – two spaces for indention
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r105812230

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.common
          +
          +import org.apache.calcite.plan.RelOptRuleCall
          +import org.apache.calcite.rel.core.

          {Calc, TableScan}

          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.dataset.DataSetCalc
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.plan.util.RexProgramExtractor
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +trait PushFilterIntoTableSourceScanRuleBase {
          +
          + private[flink] def pushFilterIntoScan(
          + call: RelOptRuleCall,
          + calc: Calc,
          + scan: TableScan,
          + tableSourceTable: TableSourceTable[_],
          + filterableSource: FilterableTableSource,
          + description: String): Unit = {
          +
          + if (filterableSource.isFilterPushedDown)

          { + // The rule can get triggered again due to the transformed "scan => filter" + // sequence created by the earlier execution of this rule when we could not + // push all the conditions into the scan + return + }

          +
          + val program = calc.getProgram
          + val (predicates, unconvertedRexNodes) =
          + RexProgramExtractor.extractConjunctiveConditions(
          + program,
          + call.builder().getRexBuilder,
          + tableSourceTable.tableEnv.getFunctionCatalog)
          + if (predicates.isEmpty)

          { + // no condition can be translated to expression + return + }

          +
          + // trying to apply filter push down, set the flag to true no matter whether
          + // we actually push any filters down.
          + filterableSource.setFilterPushedDown(true)
          + val remainingPredicates = filterableSource.applyPredicate(predicates)
          +
          + // check whether framework still need to do a filter
          + val relBuilder = call.builder()
          + val remainingCondition = {
          + if (remainingPredicates.length > 0 || unconvertedRexNodes.length > 0) {
          — End diff –

          nonEmpty is better than length > 0, i think

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r105812230 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.common + +import org.apache.calcite.plan.RelOptRuleCall +import org.apache.calcite.rel.core. {Calc, TableScan} +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.dataset.DataSetCalc +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.util.RexProgramExtractor +import org.apache.flink.table.sources.FilterableTableSource + +trait PushFilterIntoTableSourceScanRuleBase { + + private [flink] def pushFilterIntoScan( + call: RelOptRuleCall, + calc: Calc, + scan: TableScan, + tableSourceTable: TableSourceTable [_] , + filterableSource: FilterableTableSource, + description: String): Unit = { + + if (filterableSource.isFilterPushedDown) { + // The rule can get triggered again due to the transformed "scan => filter" + // sequence created by the earlier execution of this rule when we could not + // push all the conditions into the scan + return + } + + val program = calc.getProgram + val (predicates, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + call.builder().getRexBuilder, + tableSourceTable.tableEnv.getFunctionCatalog) + if (predicates.isEmpty) { + // no condition can be translated to expression + return + } + + // trying to apply filter push down, set the flag to true no matter whether + // we actually push any filters down. + filterableSource.setFilterPushedDown(true) + val remainingPredicates = filterableSource.applyPredicate(predicates) + + // check whether framework still need to do a filter + val relBuilder = call.builder() + val remainingCondition = { + if (remainingPredicates.length > 0 || unconvertedRexNodes.length > 0) { — End diff – nonEmpty is better than length > 0, i think
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r105814436

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.common
          +
          +import org.apache.calcite.plan.RelOptRuleCall
          +import org.apache.calcite.rel.core.

          {Calc, TableScan}

          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.dataset.DataSetCalc
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.plan.util.RexProgramExtractor
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +trait PushFilterIntoTableSourceScanRuleBase {
          +
          + private[flink] def pushFilterIntoScan(
          + call: RelOptRuleCall,
          + calc: Calc,
          + scan: TableScan,
          + tableSourceTable: TableSourceTable[_],
          + filterableSource: FilterableTableSource,
          + description: String): Unit = {
          +
          + if (filterableSource.isFilterPushedDown) {
          — End diff –

          tableSource should do not share between TableScan instances

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r105814436 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.common + +import org.apache.calcite.plan.RelOptRuleCall +import org.apache.calcite.rel.core. {Calc, TableScan} +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.dataset.DataSetCalc +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.util.RexProgramExtractor +import org.apache.flink.table.sources.FilterableTableSource + +trait PushFilterIntoTableSourceScanRuleBase { + + private [flink] def pushFilterIntoScan( + call: RelOptRuleCall, + calc: Calc, + scan: TableScan, + tableSourceTable: TableSourceTable [_] , + filterableSource: FilterableTableSource, + description: String): Unit = { + + if (filterableSource.isFilterPushedDown) { — End diff – tableSource should do not share between TableScan instances
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user KurtYoung commented on the issue:

          https://github.com/apache/flink/pull/3520

          To address the problem of not reusing `TableSource` when we create a new scan for table source, i changed some inheritance for current `BatchScan`, `StreamScan`, `BatchTableSourceScan`, `StreamTableSourceScan` and so on. The new structure is moe likely as the relationship between `FlinkTable` and `TableSourceTable`, `DataSetTable`, `DataStreamTable`.

          After changing the structure, it also make it possible to unify the push project into scan rule for both batch and stream mode.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on the issue: https://github.com/apache/flink/pull/3520 To address the problem of not reusing `TableSource` when we create a new scan for table source, i changed some inheritance for current `BatchScan`, `StreamScan`, `BatchTableSourceScan`, `StreamTableSourceScan` and so on. The new structure is moe likely as the relationship between `FlinkTable` and `TableSourceTable`, `DataSetTable`, `DataStreamTable`. After changing the structure, it also make it possible to unify the push project into scan rule for both batch and stream mode.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106073030

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala —
          @@ -0,0 +1,184 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlFunction, SqlPostfixOperator}

          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.util.

          {Failure, Success, Try}

          +
          +object RexProgramExtractor {
          +
          + /**
          + * Extracts the indices of input fields which accessed by the RexProgram.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return The indices of accessed input fields
          + */
          + def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
          + val visitor = new InputRefVisitor
          +
          + // extract referenced input fields from projections
          + rexProgram.getProjectList.foreach(
          + exp => rexProgram.expandLocalRef(exp).accept(visitor))
          +
          + // extract referenced input fields from condition
          + val condition = rexProgram.getCondition
          + if (condition != null)

          { + rexProgram.expandLocalRef(condition).accept(visitor) + }

          +
          + visitor.getFields
          + }
          +
          + /**
          + * Extract condition from RexProgram and convert it into independent CNF expressions.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expressions as well as RexNodes which cannot be translated
          + */
          + def extractConjunctiveConditions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = {
          +
          + rexProgram.getCondition match {
          + case condition: RexLocalRef =>
          + val expanded = rexProgram.expandLocalRef(condition)
          + // converts the expanded expression to conjunctive normal form,
          + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)"
          + val cnf = RexUtil.toCnf(rexBuilder, expanded)
          + // converts the cnf condition to a list of AND conditions
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          +
          + val convertedExpressions = new mutable.ArrayBuffer[Expression]
          + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode]
          + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray
          + val converter = new ConvertToExpression(inputNames, catalog)
          +
          + conjunctions.asScala.foreach(rex => {
          + rex.accept(converter) match

          { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + }

          + })
          + (convertedExpressions.toArray, unconvertedRexNodes.toArray)
          +
          + case _ => (Array.empty, Array.empty)
          + }
          + }
          +}
          +
          +/**
          + * An RexVisitor to extract all referenced input fields
          + */
          +class InputRefVisitor extends RexVisitorImpl[Unit](true)

          { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +}

          +
          +/**
          + * An RexVisitor to convert RexNode to Expression.
          + *
          + * @param inputNames The input names of the relation node
          + * @param functionCatalog The function catalog
          + */
          +class ConvertToExpression(
          + inputNames: Array[String],
          + functionCatalog: FunctionCatalog)
          + extends RexVisitor[Option[Expression]] {
          +
          + override def visitInputRef(inputRef: RexInputRef): Option[Expression] =

          { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + }

          +
          + override def visitLocalRef(localRef: RexLocalRef): Option[Expression] =

          { + throw new TableException("Bug: RefLocalRef should have been expanded") + }

          +
          + override def visitLiteral(literal: RexLiteral): Option[Expression] =

          { + Option(Literal(literal.getValue, FlinkTypeFactory.toTypeInfo(literal.getType))) + }

          +
          + override def visitCall(call: RexCall): Option[Expression] = {
          + val operands = call.getOperands.map(
          + operand => operand.accept(this).orNull
          + )
          +
          + // return null if we cannot translate all the operands of the call
          + if (operands.contains(null)) {
          + Option.empty
          — End diff –

          I think, `None` is more brief than `Option.empty`

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106073030 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala — @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlFunction, SqlPostfixOperator} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util. {Failure, Success, Try} + +object RexProgramExtractor { + + /** + * Extracts the indices of input fields which accessed by the RexProgram. + * + * @param rexProgram The RexProgram to analyze + * @return The indices of accessed input fields + */ + def extractRefInputFields(rexProgram: RexProgram): Array [Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + rexProgram.getProjectList.foreach( + exp => rexProgram.expandLocalRef(exp).accept(visitor)) + + // extract referenced input fields from condition + val condition = rexProgram.getCondition + if (condition != null) { + rexProgram.expandLocalRef(condition).accept(visitor) + } + + visitor.getFields + } + + /** + * Extract condition from RexProgram and convert it into independent CNF expressions. + * + * @param rexProgram The RexProgram to analyze + * @return converted expressions as well as RexNodes which cannot be translated + */ + def extractConjunctiveConditions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): (Array [Expression] , Array [RexNode] ) = { + + rexProgram.getCondition match { + case condition: RexLocalRef => + val expanded = rexProgram.expandLocalRef(condition) + // converts the expanded expression to conjunctive normal form, + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" + val cnf = RexUtil.toCnf(rexBuilder, expanded) + // converts the cnf condition to a list of AND conditions + val conjunctions = RelOptUtil.conjunctions(cnf) + + val convertedExpressions = new mutable.ArrayBuffer [Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer [RexNode] + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray + val converter = new ConvertToExpression(inputNames, catalog) + + conjunctions.asScala.foreach(rex => { + rex.accept(converter) match { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + } + }) + (convertedExpressions.toArray, unconvertedRexNodes.toArray) + + case _ => (Array.empty, Array.empty) + } + } +} + +/** + * An RexVisitor to extract all referenced input fields + */ +class InputRefVisitor extends RexVisitorImpl [Unit] (true) { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +} + +/** + * An RexVisitor to convert RexNode to Expression. + * + * @param inputNames The input names of the relation node + * @param functionCatalog The function catalog + */ +class ConvertToExpression( + inputNames: Array [String] , + functionCatalog: FunctionCatalog) + extends RexVisitor[Option [Expression] ] { + + override def visitInputRef(inputRef: RexInputRef): Option [Expression] = { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + } + + override def visitLocalRef(localRef: RexLocalRef): Option [Expression] = { + throw new TableException("Bug: RefLocalRef should have been expanded") + } + + override def visitLiteral(literal: RexLiteral): Option [Expression] = { + Option(Literal(literal.getValue, FlinkTypeFactory.toTypeInfo(literal.getType))) + } + + override def visitCall(call: RexCall): Option [Expression] = { + val operands = call.getOperands.map( + operand => operand.accept(this).orNull + ) + + // return null if we cannot translate all the operands of the call + if (operands.contains(null)) { + Option.empty — End diff – I think, `None` is more brief than `Option.empty`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106072764

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala —
          @@ -0,0 +1,184 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlFunction, SqlPostfixOperator}

          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.util.

          {Failure, Success, Try}

          +
          +object RexProgramExtractor {
          +
          + /**
          + * Extracts the indices of input fields which accessed by the RexProgram.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return The indices of accessed input fields
          + */
          + def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
          + val visitor = new InputRefVisitor
          +
          + // extract referenced input fields from projections
          + rexProgram.getProjectList.foreach(
          + exp => rexProgram.expandLocalRef(exp).accept(visitor))
          +
          + // extract referenced input fields from condition
          + val condition = rexProgram.getCondition
          + if (condition != null)

          { + rexProgram.expandLocalRef(condition).accept(visitor) + }

          +
          + visitor.getFields
          + }
          +
          + /**
          + * Extract condition from RexProgram and convert it into independent CNF expressions.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expressions as well as RexNodes which cannot be translated
          + */
          + def extractConjunctiveConditions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = {
          +
          + rexProgram.getCondition match {
          + case condition: RexLocalRef =>
          + val expanded = rexProgram.expandLocalRef(condition)
          + // converts the expanded expression to conjunctive normal form,
          + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)"
          + val cnf = RexUtil.toCnf(rexBuilder, expanded)
          + // converts the cnf condition to a list of AND conditions
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          +
          + val convertedExpressions = new mutable.ArrayBuffer[Expression]
          + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode]
          + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray
          + val converter = new ConvertToExpression(inputNames, catalog)
          +
          + conjunctions.asScala.foreach(rex => {
          + rex.accept(converter) match

          { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + }

          + })
          + (convertedExpressions.toArray, unconvertedRexNodes.toArray)
          +
          + case _ => (Array.empty, Array.empty)
          + }
          + }
          +}
          +
          +/**
          + * An RexVisitor to extract all referenced input fields
          + */
          +class InputRefVisitor extends RexVisitorImpl[Unit](true) {
          +
          + private var fields = mutable.LinkedHashSet[Int]()
          — End diff –

          `var` → `val`

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106072764 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala — @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlFunction, SqlPostfixOperator} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util. {Failure, Success, Try} + +object RexProgramExtractor { + + /** + * Extracts the indices of input fields which accessed by the RexProgram. + * + * @param rexProgram The RexProgram to analyze + * @return The indices of accessed input fields + */ + def extractRefInputFields(rexProgram: RexProgram): Array [Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + rexProgram.getProjectList.foreach( + exp => rexProgram.expandLocalRef(exp).accept(visitor)) + + // extract referenced input fields from condition + val condition = rexProgram.getCondition + if (condition != null) { + rexProgram.expandLocalRef(condition).accept(visitor) + } + + visitor.getFields + } + + /** + * Extract condition from RexProgram and convert it into independent CNF expressions. + * + * @param rexProgram The RexProgram to analyze + * @return converted expressions as well as RexNodes which cannot be translated + */ + def extractConjunctiveConditions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): (Array [Expression] , Array [RexNode] ) = { + + rexProgram.getCondition match { + case condition: RexLocalRef => + val expanded = rexProgram.expandLocalRef(condition) + // converts the expanded expression to conjunctive normal form, + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" + val cnf = RexUtil.toCnf(rexBuilder, expanded) + // converts the cnf condition to a list of AND conditions + val conjunctions = RelOptUtil.conjunctions(cnf) + + val convertedExpressions = new mutable.ArrayBuffer [Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer [RexNode] + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray + val converter = new ConvertToExpression(inputNames, catalog) + + conjunctions.asScala.foreach(rex => { + rex.accept(converter) match { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + } + }) + (convertedExpressions.toArray, unconvertedRexNodes.toArray) + + case _ => (Array.empty, Array.empty) + } + } +} + +/** + * An RexVisitor to extract all referenced input fields + */ +class InputRefVisitor extends RexVisitorImpl [Unit] (true) { + + private var fields = mutable.LinkedHashSet [Int] () — End diff – `var` → `val`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106074574

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala —
          @@ -19,31 +19,34 @@
          package org.apache.flink.table.plan.nodes.dataset

          import org.apache.calcite.plan._
          +import org.apache.calcite.rel.`type`.RelDataType
          import org.apache.calcite.rel.metadata.RelMetadataQuery
          import org.apache.calcite.rel.

          {RelNode, RelWriter}

          import org.apache.flink.api.java.DataSet
          import org.apache.flink.table.api.

          {BatchTableEnvironment, TableEnvironment}

          import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.plan.nodes.TableSourceScan
          import org.apache.flink.table.plan.schema.TableSourceTable
          -import org.apache.flink.table.sources.BatchTableSource
          +import org.apache.flink.table.sources.

          {BatchTableSource, TableSource}

          import org.apache.flink.types.Row

          /** Flink RelNode to read data from an external source defined by a [[BatchTableSource]]. */
          class BatchTableSourceScan(
          cluster: RelOptCluster,
          traitSet: RelTraitSet,
          table: RelOptTable,

          • val tableSource: BatchTableSource[_])
          • extends BatchScan(cluster, traitSet, table) {
            + tableSource: BatchTableSource[_])
            + extends TableSourceScan(cluster, traitSet, table, tableSource)
            + with BatchScan {
          • override def deriveRowType() = {
            + override def deriveRowType(): RelDataType = {
              • End diff –

          Move this method to `TableSourceScan`. The behavior is same between `TableSourceScan`'s sub-class.

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106074574 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala — @@ -19,31 +19,34 @@ package org.apache.flink.table.plan.nodes.dataset import org.apache.calcite.plan._ +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel. {RelNode, RelWriter} import org.apache.flink.api.java.DataSet import org.apache.flink.table.api. {BatchTableEnvironment, TableEnvironment} import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.plan.nodes.TableSourceScan import org.apache.flink.table.plan.schema.TableSourceTable -import org.apache.flink.table.sources.BatchTableSource +import org.apache.flink.table.sources. {BatchTableSource, TableSource} import org.apache.flink.types.Row /** Flink RelNode to read data from an external source defined by a [ [BatchTableSource] ]. */ class BatchTableSourceScan( cluster: RelOptCluster, traitSet: RelTraitSet, table: RelOptTable, val tableSource: BatchTableSource [_] ) extends BatchScan(cluster, traitSet, table) { + tableSource: BatchTableSource [_] ) + extends TableSourceScan(cluster, traitSet, table, tableSource) + with BatchScan { override def deriveRowType() = { + override def deriveRowType(): RelDataType = { End diff – Move this method to `TableSourceScan`. The behavior is same between `TableSourceScan`'s sub-class.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106100087

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala —
          @@ -57,17 +60,24 @@ class BatchTableSourceScan(
          )
          }

          + override def copy(traitSet: RelTraitSet, newTableSource: TableSource[_]): TableSourceScan = {
          + new BatchTableSourceScan(
          + cluster,
          + traitSet,
          + getTable,
          + newTableSource.asInstanceOf[BatchTableSource[_]]
          — End diff –

          `TableSource` instance in `TableSourceTable` which can be get by `getTable.unwrap(classOf[TableSourceTable[_]])` has not changed never, and it may be different from newTableSource.

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106100087 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala — @@ -57,17 +60,24 @@ class BatchTableSourceScan( ) } + override def copy(traitSet: RelTraitSet, newTableSource: TableSource [_] ): TableSourceScan = { + new BatchTableSourceScan( + cluster, + traitSet, + getTable, + newTableSource.asInstanceOf[BatchTableSource [_] ] — End diff – `TableSource` instance in `TableSourceTable` which can be get by `getTable.unwrap(classOf[TableSourceTable [_] ])` has not changed never, and it may be different from newTableSource.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106149747

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -34,14 +34,24 @@ trait FilterableTableSource {

          /**

          • Check and pick all predicates this table source can support. The passed in predicates
          • * have been translated in conjunctive form, and table source and only pick those predicates
          • * that it supports. All unsupported predicates should be give back to the framework to do
          • * further filtering.
            + * have been translated in conjunctive form, and table source can only pick those predicates
            + * that it supports.
            + * <p>
            + * After trying to push predicates down, we should return a new [[FilterableTableSource]]
            + * instance which holds all pushed down predicates. Even if we actually pushed nothing down,
            + * it is recommended that we still return a new [[FilterableTableSource]] instance since we will
            + * mark the returned instance as filter push down has been tried. Also we need to return all
            + * unsupported predicates back to the framework to do further filtering.
            + * <p>
            + * We also should note to not changing the form of the predicates passed in. It has been
            + * organized in CNF conjunctive form, and we should only take or leave each element in the
            + * array. Don't try to reorganize the predicates if you are absolutely confident with that.
            *
          • @param predicate An array contains conjunctive predicates.
          • * @return An array contains all unsupported predicates.
            + * @return A new cloned instance of [[FilterableTableSource]] as well as n array of Expression
            + * which contains all unsupported predicates.
            */
          • def applyPredicate(predicate: Array[Expression]): Array[Expression]
            + def applyPredicate(predicate: Array[Expression]): (FilterableTableSource[_], Array[Expression])
              • End diff –

          return `FilterableTableSource[T]` instead of `FilterableTableSource[_]`, otherwise in java, table source instance must be cast to `FilterableTableSource<?>`

          Show
          githubbot ASF GitHub Bot added a comment - Github user godfreyhe commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106149747 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -34,14 +34,24 @@ trait FilterableTableSource { /** Check and pick all predicates this table source can support. The passed in predicates * have been translated in conjunctive form, and table source and only pick those predicates * that it supports. All unsupported predicates should be give back to the framework to do * further filtering. + * have been translated in conjunctive form, and table source can only pick those predicates + * that it supports. + * <p> + * After trying to push predicates down, we should return a new [ [FilterableTableSource] ] + * instance which holds all pushed down predicates. Even if we actually pushed nothing down, + * it is recommended that we still return a new [ [FilterableTableSource] ] instance since we will + * mark the returned instance as filter push down has been tried. Also we need to return all + * unsupported predicates back to the framework to do further filtering. + * <p> + * We also should note to not changing the form of the predicates passed in. It has been + * organized in CNF conjunctive form, and we should only take or leave each element in the + * array. Don't try to reorganize the predicates if you are absolutely confident with that. * @param predicate An array contains conjunctive predicates. * @return An array contains all unsupported predicates. + * @return A new cloned instance of [ [FilterableTableSource] ] as well as n array of Expression + * which contains all unsupported predicates. */ def applyPredicate(predicate: Array [Expression] ): Array [Expression] + def applyPredicate(predicate: Array [Expression] ): (FilterableTableSource [_] , Array [Expression] ) End diff – return `FilterableTableSource [T] ` instead of `FilterableTableSource [_] `, otherwise in java, table source instance must be cast to `FilterableTableSource<?>`
          Hide
          githubbot ASF GitHub Bot added a comment -

          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`?

          Show
          githubbot ASF GitHub Bot added a comment - 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`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106137246

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.common
          +
          +import org.apache.calcite.plan.RelOptRuleCall
          +import org.apache.calcite.rel.core.Calc
          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.TableSourceScan
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.plan.util.RexProgramExtractor
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +trait PushFilterIntoTableSourceScanRuleBase {
          +
          + private[flink] def pushFilterIntoScan(
          + call: RelOptRuleCall,
          + calc: Calc,
          + scan: TableSourceScan,
          + tableSourceTable: TableSourceTable[_],
          + filterableSource: FilterableTableSource[_],
          + description: String): Unit = {
          +
          + if (filterableSource.isFilterPushedDown)

          { + // The rule can get triggered again due to the transformed "scan => filter" + // sequence created by the earlier execution of this rule when we could not + // push all the conditions into the scan + return + }

          +
          + val program = calc.getProgram
          + val (predicates, unconvertedRexNodes) =
          + RexProgramExtractor.extractConjunctiveConditions(
          + program,
          + call.builder().getRexBuilder,
          + tableSourceTable.tableEnv.getFunctionCatalog)
          + if (predicates.isEmpty)

          { + // no condition can be translated to expression + return + }

          +
          + val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates)
          + // trying to apply filter push down, set the flag to true no matter whether
          + // we actually push any filters down.
          + newTableSource.setFilterPushedDown(true)
          +
          + // check whether framework still need to do a filter
          + val relBuilder = call.builder()
          + val remainingCondition = {
          + if (remainingPredicates.nonEmpty || unconvertedRexNodes.nonEmpty)

          { + relBuilder.push(scan) + (remainingPredicates.map(expr => expr.toRexNode(relBuilder)) ++ unconvertedRexNodes) + .reduce((l, r) => relBuilder.and(l, r)) + }

          else

          { + null + }

          + }
          +
          + // check whether we still need a RexProgram. An RexProgram is needed when either
          + // projection or filter exists.
          + val newScan = scan.copy(scan.getTraitSet, newTableSource)
          + val newRexProgram = {
          + if (remainingCondition != null || program.getProjectList.size() > 0) {
          — End diff –

          `program.getProjectList.size() > 0` should be `program.projectsOnlyIdentity()`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106137246 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.common + +import org.apache.calcite.plan.RelOptRuleCall +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.TableSourceScan +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.util.RexProgramExtractor +import org.apache.flink.table.sources.FilterableTableSource + +trait PushFilterIntoTableSourceScanRuleBase { + + private [flink] def pushFilterIntoScan( + call: RelOptRuleCall, + calc: Calc, + scan: TableSourceScan, + tableSourceTable: TableSourceTable [_] , + filterableSource: FilterableTableSource [_] , + description: String): Unit = { + + if (filterableSource.isFilterPushedDown) { + // The rule can get triggered again due to the transformed "scan => filter" + // sequence created by the earlier execution of this rule when we could not + // push all the conditions into the scan + return + } + + val program = calc.getProgram + val (predicates, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + call.builder().getRexBuilder, + tableSourceTable.tableEnv.getFunctionCatalog) + if (predicates.isEmpty) { + // no condition can be translated to expression + return + } + + val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates) + // trying to apply filter push down, set the flag to true no matter whether + // we actually push any filters down. + newTableSource.setFilterPushedDown(true) + + // check whether framework still need to do a filter + val relBuilder = call.builder() + val remainingCondition = { + if (remainingPredicates.nonEmpty || unconvertedRexNodes.nonEmpty) { + relBuilder.push(scan) + (remainingPredicates.map(expr => expr.toRexNode(relBuilder)) ++ unconvertedRexNodes) + .reduce((l, r) => relBuilder.and(l, r)) + } else { + null + } + } + + // check whether we still need a RexProgram. An RexProgram is needed when either + // projection or filter exists. + val newScan = scan.copy(scan.getTraitSet, newTableSource) + val newRexProgram = { + if (remainingCondition != null || program.getProjectList.size() > 0) { — End diff – `program.getProjectList.size() > 0` should be `program.projectsOnlyIdentity()`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106123284

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala —
          @@ -25,6 +25,7 @@ import org.apache.flink.table.sources.TableSource
          /** Table which defines an external table via a [[TableSource]] */
          class TableSourceTable[T](
          val tableSource: TableSource[T],
          + val tableEnv: TableEnvironment,
          — End diff –

          Do we really need this reference here? I think it is only needs to provide the FunctionCatalog for translating RexNodes into Expressions for filter pushdown. Isn't the catalog of built-in functions sufficient for that (which is available as a static object)? I don't think a TableSource would be able to evaluate a predicate that includes a UDF, so the built-in functions should be enough and we do not need to add the `TableEnvironment` here.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106123284 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala — @@ -25,6 +25,7 @@ import org.apache.flink.table.sources.TableSource /** Table which defines an external table via a [ [TableSource] ] */ class TableSourceTable [T] ( val tableSource: TableSource [T] , + val tableEnv: TableEnvironment, — End diff – Do we really need this reference here? I think it is only needs to provide the FunctionCatalog for translating RexNodes into Expressions for filter pushdown. Isn't the catalog of built-in functions sufficient for that (which is available as a static object)? I don't think a TableSource would be able to evaluate a predicate that includes a UDF, so the built-in functions should be enough and we do not need to add the `TableEnvironment` here.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106031866

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala —
          @@ -58,16 +60,24 @@ class BatchTableSourceScan(
          )
          }

          + override def copy(traitSet: RelTraitSet, newTableSource: TableSource[_]): TableSourceScan =

          { + new BatchTableSourceScan( + cluster, + traitSet, + getTable, + newTableSource.asInstanceOf[BatchTableSource[_]] + ) + }

          +
          override def explainTerms(pw: RelWriter): RelWriter = {

          • super.explainTerms(pw)
            + val terms = super.explainTerms(pw)
            .item("fields", TableEnvironment.getFieldNames(tableSource).mkString(", "))
            + tableSource.explainTerms(terms)
              • End diff –

          Insert string provided by `TableSource.toString()` (or a dedicated method that returns an explain string) instead of calling `explainTerms()`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106031866 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala — @@ -58,16 +60,24 @@ class BatchTableSourceScan( ) } + override def copy(traitSet: RelTraitSet, newTableSource: TableSource [_] ): TableSourceScan = { + new BatchTableSourceScan( + cluster, + traitSet, + getTable, + newTableSource.asInstanceOf[BatchTableSource[_]] + ) + } + override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) + val terms = super.explainTerms(pw) .item("fields", TableEnvironment.getFieldNames(tableSource).mkString(", ")) + tableSource.explainTerms(terms) End diff – Insert string provided by `TableSource.toString()` (or a dedicated method that returns an explain string) instead of calling `explainTerms()`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106140096

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/ProjectableTableSource.scala —
          @@ -24,7 +24,7 @@ package org.apache.flink.table.sources
          *

          • @tparam T The return type of the [[ProjectableTableSource]].
            */
            -trait ProjectableTableSource[T] {
            +trait ProjectableTableSource[T] extends TableSource[T] {
              • End diff –

          Why is this change necessary? A `ProjectableTableSource` always needs to implement either `BatchTableSource` or `StreamTableSource` (or both) which already implement `TableSource`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106140096 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/ProjectableTableSource.scala — @@ -24,7 +24,7 @@ package org.apache.flink.table.sources * @tparam T The return type of the [ [ProjectableTableSource] ]. */ -trait ProjectableTableSource [T] { +trait ProjectableTableSource [T] extends TableSource [T] { End diff – Why is this change necessary? A `ProjectableTableSource` always needs to implement either `BatchTableSource` or `StreamTableSource` (or both) which already implement `TableSource`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106032363

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala —
          @@ -42,63 +41,40 @@ class DataSetCalc(
          traitSet: RelTraitSet,
          input: RelNode,
          rowRelDataType: RelDataType,

          • private[flink] val calcProgram: RexProgram, // for tests
            + calcProgram: RexProgram,
            ruleDescription: String)
          • extends SingleRel(cluster, traitSet, input)
            + extends Calc(cluster, traitSet, input, calcProgram)
              • End diff –

          Why are you changing this?
          I think we had this at some point in time, but changed `Calc` to `SingleRel`. Can't remember why though. Maybe @twalthr knows why.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106032363 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala — @@ -42,63 +41,40 @@ class DataSetCalc( traitSet: RelTraitSet, input: RelNode, rowRelDataType: RelDataType, private [flink] val calcProgram: RexProgram, // for tests + calcProgram: RexProgram, ruleDescription: String) extends SingleRel(cluster, traitSet, input) + extends Calc(cluster, traitSet, input, calcProgram) End diff – Why are you changing this? I think we had this at some point in time, but changed `Calc` to `SingleRel`. Can't remember why though. Maybe @twalthr knows why.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106159736

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala —
          @@ -0,0 +1,184 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlFunction, SqlPostfixOperator}

          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.util.

          {Failure, Success, Try}

          +
          +object RexProgramExtractor {
          +
          + /**
          + * Extracts the indices of input fields which accessed by the RexProgram.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return The indices of accessed input fields
          + */
          + def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
          + val visitor = new InputRefVisitor
          +
          + // extract referenced input fields from projections
          + rexProgram.getProjectList.foreach(
          + exp => rexProgram.expandLocalRef(exp).accept(visitor))
          +
          + // extract referenced input fields from condition
          + val condition = rexProgram.getCondition
          + if (condition != null)

          { + rexProgram.expandLocalRef(condition).accept(visitor) + }

          +
          + visitor.getFields
          + }
          +
          + /**
          + * Extract condition from RexProgram and convert it into independent CNF expressions.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expressions as well as RexNodes which cannot be translated
          + */
          + def extractConjunctiveConditions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = {
          +
          + rexProgram.getCondition match {
          + case condition: RexLocalRef =>
          + val expanded = rexProgram.expandLocalRef(condition)
          + // converts the expanded expression to conjunctive normal form,
          + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)"
          + val cnf = RexUtil.toCnf(rexBuilder, expanded)
          + // converts the cnf condition to a list of AND conditions
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          +
          + val convertedExpressions = new mutable.ArrayBuffer[Expression]
          + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode]
          + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray
          + val converter = new ConvertToExpression(inputNames, catalog)
          +
          + conjunctions.asScala.foreach(rex => {
          + rex.accept(converter) match

          { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + }

          + })
          + (convertedExpressions.toArray, unconvertedRexNodes.toArray)
          +
          + case _ => (Array.empty, Array.empty)
          + }
          + }
          +}
          +
          +/**
          + * An RexVisitor to extract all referenced input fields
          + */
          +class InputRefVisitor extends RexVisitorImpl[Unit](true)

          { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +}

          +
          +/**
          + * An RexVisitor to convert RexNode to Expression.
          + *
          + * @param inputNames The input names of the relation node
          + * @param functionCatalog The function catalog
          + */
          +class ConvertToExpression(
          + inputNames: Array[String],
          + functionCatalog: FunctionCatalog)
          + extends RexVisitor[Option[Expression]] {
          +
          + override def visitInputRef(inputRef: RexInputRef): Option[Expression] =

          { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + }

          +
          + override def visitLocalRef(localRef: RexLocalRef): Option[Expression] = {
          + throw new TableException("Bug: RefLocalRef should have been expanded")
          — End diff –

          `RefLocalRef` -> `RexLocalRef`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106159736 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala — @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlFunction, SqlPostfixOperator} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util. {Failure, Success, Try} + +object RexProgramExtractor { + + /** + * Extracts the indices of input fields which accessed by the RexProgram. + * + * @param rexProgram The RexProgram to analyze + * @return The indices of accessed input fields + */ + def extractRefInputFields(rexProgram: RexProgram): Array [Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + rexProgram.getProjectList.foreach( + exp => rexProgram.expandLocalRef(exp).accept(visitor)) + + // extract referenced input fields from condition + val condition = rexProgram.getCondition + if (condition != null) { + rexProgram.expandLocalRef(condition).accept(visitor) + } + + visitor.getFields + } + + /** + * Extract condition from RexProgram and convert it into independent CNF expressions. + * + * @param rexProgram The RexProgram to analyze + * @return converted expressions as well as RexNodes which cannot be translated + */ + def extractConjunctiveConditions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): (Array [Expression] , Array [RexNode] ) = { + + rexProgram.getCondition match { + case condition: RexLocalRef => + val expanded = rexProgram.expandLocalRef(condition) + // converts the expanded expression to conjunctive normal form, + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" + val cnf = RexUtil.toCnf(rexBuilder, expanded) + // converts the cnf condition to a list of AND conditions + val conjunctions = RelOptUtil.conjunctions(cnf) + + val convertedExpressions = new mutable.ArrayBuffer [Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer [RexNode] + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray + val converter = new ConvertToExpression(inputNames, catalog) + + conjunctions.asScala.foreach(rex => { + rex.accept(converter) match { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + } + }) + (convertedExpressions.toArray, unconvertedRexNodes.toArray) + + case _ => (Array.empty, Array.empty) + } + } +} + +/** + * An RexVisitor to extract all referenced input fields + */ +class InputRefVisitor extends RexVisitorImpl [Unit] (true) { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +} + +/** + * An RexVisitor to convert RexNode to Expression. + * + * @param inputNames The input names of the relation node + * @param functionCatalog The function catalog + */ +class ConvertToExpression( + inputNames: Array [String] , + functionCatalog: FunctionCatalog) + extends RexVisitor[Option [Expression] ] { + + override def visitInputRef(inputRef: RexInputRef): Option [Expression] = { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + } + + override def visitLocalRef(localRef: RexLocalRef): Option [Expression] = { + throw new TableException("Bug: RefLocalRef should have been expanded") — End diff – `RefLocalRef` -> `RexLocalRef`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106123397

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala —
          @@ -0,0 +1,184 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlFunction, SqlPostfixOperator}

          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.util.

          {Failure, Success, Try}

          +
          +object RexProgramExtractor {
          +
          + /**
          + * Extracts the indices of input fields which accessed by the RexProgram.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return The indices of accessed input fields
          + */
          + def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
          + val visitor = new InputRefVisitor
          +
          + // extract referenced input fields from projections
          + rexProgram.getProjectList.foreach(
          + exp => rexProgram.expandLocalRef(exp).accept(visitor))
          +
          + // extract referenced input fields from condition
          + val condition = rexProgram.getCondition
          + if (condition != null)

          { + rexProgram.expandLocalRef(condition).accept(visitor) + }

          +
          + visitor.getFields
          + }
          +
          + /**
          + * Extract condition from RexProgram and convert it into independent CNF expressions.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expressions as well as RexNodes which cannot be translated
          + */
          + def extractConjunctiveConditions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = {
          +
          + rexProgram.getCondition match {
          + case condition: RexLocalRef =>
          + val expanded = rexProgram.expandLocalRef(condition)
          + // converts the expanded expression to conjunctive normal form,
          + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)"
          + val cnf = RexUtil.toCnf(rexBuilder, expanded)
          + // converts the cnf condition to a list of AND conditions
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          +
          + val convertedExpressions = new mutable.ArrayBuffer[Expression]
          + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode]
          + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray
          + val converter = new ConvertToExpression(inputNames, catalog)
          +
          + conjunctions.asScala.foreach(rex => {
          + rex.accept(converter) match

          { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + }

          + })
          + (convertedExpressions.toArray, unconvertedRexNodes.toArray)
          +
          + case _ => (Array.empty, Array.empty)
          + }
          + }
          +}
          +
          +/**
          + * An RexVisitor to extract all referenced input fields
          + */
          +class InputRefVisitor extends RexVisitorImpl[Unit](true)

          { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +}

          +
          +/**
          + * An RexVisitor to convert RexNode to Expression.
          + *
          + * @param inputNames The input names of the relation node
          + * @param functionCatalog The function catalog
          + */
          +class ConvertToExpression(
          + inputNames: Array[String],
          + functionCatalog: FunctionCatalog)
          + extends RexVisitor[Option[Expression]] {
          +
          + override def visitInputRef(inputRef: RexInputRef): Option[Expression] =

          { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + }

          +
          + override def visitLocalRef(localRef: RexLocalRef): Option[Expression] =

          { + throw new TableException("Bug: RefLocalRef should have been expanded") + }

          +
          + override def visitLiteral(literal: RexLiteral): Option[Expression] =

          { + Option(Literal(literal.getValue, FlinkTypeFactory.toTypeInfo(literal.getType))) + }

          +
          + override def visitCall(call: RexCall): Option[Expression] = {
          + val operands = call.getOperands.map(
          + operand => operand.accept(this).orNull
          + )
          +
          + // return null if we cannot translate all the operands of the call
          + if (operands.contains(null))

          { + Option.empty + }

          else {
          + call.getOperator match {
          + case function: SqlFunction =>
          + lookupFunction(replace(function.getName), operands)
          + case postfix: SqlPostfixOperator =>
          + lookupFunction(replace(postfix.getName), operands)
          + case operator@_ =>
          + lookupFunction(replace(s"$

          {operator.getKind}

          "), operands)
          + }
          + }
          + }
          +
          + override def visitFieldAccess(fieldAccess: RexFieldAccess): Option[Expression] = Option.empty
          +
          + override def visitCorrelVariable(correlVariable: RexCorrelVariable): Option[Expression] =
          + Option.empty
          +
          + override def visitRangeRef(rangeRef: RexRangeRef): Option[Expression] = Option.empty
          +
          + override def visitSubQuery(subQuery: RexSubQuery): Option[Expression] = Option.empty
          +
          + override def visitDynamicParam(dynamicParam: RexDynamicParam): Option[Expression] = Option.empty
          +
          + override def visitOver(over: RexOver): Option[Expression] = Option.empty
          +
          + private def lookupFunction(name: String, operands: Seq[Expression]): Option[Expression] = {
          + Try(functionCatalog.lookupFunction(name, operands)) match {
          — End diff –

          Isn't the catalog of built-in functions sufficient here?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106123397 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala — @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlFunction, SqlPostfixOperator} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util. {Failure, Success, Try} + +object RexProgramExtractor { + + /** + * Extracts the indices of input fields which accessed by the RexProgram. + * + * @param rexProgram The RexProgram to analyze + * @return The indices of accessed input fields + */ + def extractRefInputFields(rexProgram: RexProgram): Array [Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + rexProgram.getProjectList.foreach( + exp => rexProgram.expandLocalRef(exp).accept(visitor)) + + // extract referenced input fields from condition + val condition = rexProgram.getCondition + if (condition != null) { + rexProgram.expandLocalRef(condition).accept(visitor) + } + + visitor.getFields + } + + /** + * Extract condition from RexProgram and convert it into independent CNF expressions. + * + * @param rexProgram The RexProgram to analyze + * @return converted expressions as well as RexNodes which cannot be translated + */ + def extractConjunctiveConditions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): (Array [Expression] , Array [RexNode] ) = { + + rexProgram.getCondition match { + case condition: RexLocalRef => + val expanded = rexProgram.expandLocalRef(condition) + // converts the expanded expression to conjunctive normal form, + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" + val cnf = RexUtil.toCnf(rexBuilder, expanded) + // converts the cnf condition to a list of AND conditions + val conjunctions = RelOptUtil.conjunctions(cnf) + + val convertedExpressions = new mutable.ArrayBuffer [Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer [RexNode] + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray + val converter = new ConvertToExpression(inputNames, catalog) + + conjunctions.asScala.foreach(rex => { + rex.accept(converter) match { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + } + }) + (convertedExpressions.toArray, unconvertedRexNodes.toArray) + + case _ => (Array.empty, Array.empty) + } + } +} + +/** + * An RexVisitor to extract all referenced input fields + */ +class InputRefVisitor extends RexVisitorImpl [Unit] (true) { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +} + +/** + * An RexVisitor to convert RexNode to Expression. + * + * @param inputNames The input names of the relation node + * @param functionCatalog The function catalog + */ +class ConvertToExpression( + inputNames: Array [String] , + functionCatalog: FunctionCatalog) + extends RexVisitor[Option [Expression] ] { + + override def visitInputRef(inputRef: RexInputRef): Option [Expression] = { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + } + + override def visitLocalRef(localRef: RexLocalRef): Option [Expression] = { + throw new TableException("Bug: RefLocalRef should have been expanded") + } + + override def visitLiteral(literal: RexLiteral): Option [Expression] = { + Option(Literal(literal.getValue, FlinkTypeFactory.toTypeInfo(literal.getType))) + } + + override def visitCall(call: RexCall): Option [Expression] = { + val operands = call.getOperands.map( + operand => operand.accept(this).orNull + ) + + // return null if we cannot translate all the operands of the call + if (operands.contains(null)) { + Option.empty + } else { + call.getOperator match { + case function: SqlFunction => + lookupFunction(replace(function.getName), operands) + case postfix: SqlPostfixOperator => + lookupFunction(replace(postfix.getName), operands) + case operator@_ => + lookupFunction(replace(s"$ {operator.getKind} "), operands) + } + } + } + + override def visitFieldAccess(fieldAccess: RexFieldAccess): Option [Expression] = Option.empty + + override def visitCorrelVariable(correlVariable: RexCorrelVariable): Option [Expression] = + Option.empty + + override def visitRangeRef(rangeRef: RexRangeRef): Option [Expression] = Option.empty + + override def visitSubQuery(subQuery: RexSubQuery): Option [Expression] = Option.empty + + override def visitDynamicParam(dynamicParam: RexDynamicParam): Option [Expression] = Option.empty + + override def visitOver(over: RexOver): Option [Expression] = Option.empty + + private def lookupFunction(name: String, operands: Seq [Expression] ): Option [Expression] = { + Try(functionCatalog.lookupFunction(name, operands)) match { — End diff – Isn't the catalog of built-in functions sufficient here?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106034668

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.common
          +
          +import org.apache.calcite.plan.RelOptRuleCall
          +import org.apache.calcite.rel.core.Calc
          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.TableSourceScan
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.plan.util.RexProgramExtractor
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +trait PushFilterIntoTableSourceScanRuleBase {
          +
          + private[flink] def pushFilterIntoScan(
          + call: RelOptRuleCall,
          + calc: Calc,
          + scan: TableSourceScan,
          + tableSourceTable: TableSourceTable[_],
          + filterableSource: FilterableTableSource[_],
          + description: String): Unit = {
          +
          + if (filterableSource.isFilterPushedDown) {
          — End diff –

          I think we should check on the `TableSourceScan` whether we tried to push a filter down.
          This check should be done in the `matches()` method and not in `onMatch()`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106034668 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.common + +import org.apache.calcite.plan.RelOptRuleCall +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.TableSourceScan +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.util.RexProgramExtractor +import org.apache.flink.table.sources.FilterableTableSource + +trait PushFilterIntoTableSourceScanRuleBase { + + private [flink] def pushFilterIntoScan( + call: RelOptRuleCall, + calc: Calc, + scan: TableSourceScan, + tableSourceTable: TableSourceTable [_] , + filterableSource: FilterableTableSource [_] , + description: String): Unit = { + + if (filterableSource.isFilterPushedDown) { — End diff – I think we should check on the `TableSourceScan` whether we tried to push a filter down. This check should be done in the `matches()` method and not in `onMatch()`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106137875

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala —
          @@ -0,0 +1,55 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.datastream
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.datastream.

          {DataStreamCalc, StreamTableSourceScan}

          +import org.apache.flink.table.plan.rules.common.PushFilterIntoTableSourceScanRuleBase
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataStreamCalc],
          + operand(classOf[StreamTableSourceScan], none)),
          + "PushFilterIntoStreamTableSourceScanRule")
          + with PushFilterIntoTableSourceScanRuleBase {
          +
          + override def matches(call: RelOptRuleCall): Boolean = {
          + val calc: DataStreamCalc = call.rel(0).asInstanceOf[DataStreamCalc]
          + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
          + scan.tableSource match {
          + case _: FilterableTableSource[_] =>
          + calc.getProgram.getCondition != null
          — End diff –

          Check if we already tried to push a filter down.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106137875 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala — @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.datastream. {DataStreamCalc, StreamTableSourceScan} +import org.apache.flink.table.plan.rules.common.PushFilterIntoTableSourceScanRuleBase +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule( + operand(classOf [DataStreamCalc] , + operand(classOf [StreamTableSourceScan] , none)), + "PushFilterIntoStreamTableSourceScanRule") + with PushFilterIntoTableSourceScanRuleBase { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: DataStreamCalc = call.rel(0).asInstanceOf [DataStreamCalc] + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf [StreamTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource [_] => + calc.getProgram.getCondition != null — End diff – Check if we already tried to push a filter down.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106137700

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala —
          @@ -0,0 +1,55 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.dataSet
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.dataset.

          {BatchTableSourceScan, DataSetCalc}

          +import org.apache.flink.table.plan.rules.common.PushFilterIntoTableSourceScanRuleBase
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataSetCalc],
          + operand(classOf[BatchTableSourceScan], none)),
          + "PushFilterIntoBatchTableSourceScanRule")
          + with PushFilterIntoTableSourceScanRuleBase {
          +
          + override def matches(call: RelOptRuleCall): Boolean = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          + scan.tableSource match {
          + case _: FilterableTableSource[_] =>
          + calc.getProgram.getCondition != null
          — End diff –

          Check also if we already tried to push a filter down

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106137700 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala — @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.dataSet + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.dataset. {BatchTableSourceScan, DataSetCalc} +import org.apache.flink.table.plan.rules.common.PushFilterIntoTableSourceScanRuleBase +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule( + operand(classOf [DataSetCalc] , + operand(classOf [BatchTableSourceScan] , none)), + "PushFilterIntoBatchTableSourceScanRule") + with PushFilterIntoTableSourceScanRuleBase { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource [_] => + calc.getProgram.getCondition != null — End diff – Check also if we already tried to push a filter down
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106140654

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -0,0 +1,66 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.sources
          +
          +import org.apache.flink.table.expressions.Expression
          +
          +/**
          + * Adds support for filtering push-down to a [[TableSource]].
          + * A [[TableSource]] extending this interface is able to filter records before returning.
          + */
          +trait FilterableTableSource[T] extends TableSource[T] {
          — End diff –

          Why do we need to extend `TableSource`? (see also comment on `ProjectableTableSource`)

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106140654 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources + +import org.apache.flink.table.expressions.Expression + +/** + * Adds support for filtering push-down to a [ [TableSource] ]. + * A [ [TableSource] ] extending this interface is able to filter records before returning. + */ +trait FilterableTableSource [T] extends TableSource [T] { — End diff – Why do we need to extend `TableSource`? (see also comment on `ProjectableTableSource`)
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106159389

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala —
          @@ -0,0 +1,184 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlFunction, SqlPostfixOperator}

          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.util.

          {Failure, Success, Try}

          +
          +object RexProgramExtractor {
          +
          + /**
          + * Extracts the indices of input fields which accessed by the RexProgram.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return The indices of accessed input fields
          + */
          + def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
          + val visitor = new InputRefVisitor
          +
          + // extract referenced input fields from projections
          + rexProgram.getProjectList.foreach(
          + exp => rexProgram.expandLocalRef(exp).accept(visitor))
          +
          + // extract referenced input fields from condition
          + val condition = rexProgram.getCondition
          + if (condition != null)

          { + rexProgram.expandLocalRef(condition).accept(visitor) + }

          +
          + visitor.getFields
          + }
          +
          + /**
          + * Extract condition from RexProgram and convert it into independent CNF expressions.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expressions as well as RexNodes which cannot be translated
          + */
          + def extractConjunctiveConditions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = {
          +
          + rexProgram.getCondition match {
          + case condition: RexLocalRef =>
          + val expanded = rexProgram.expandLocalRef(condition)
          + // converts the expanded expression to conjunctive normal form,
          + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)"
          + val cnf = RexUtil.toCnf(rexBuilder, expanded)
          + // converts the cnf condition to a list of AND conditions
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          +
          + val convertedExpressions = new mutable.ArrayBuffer[Expression]
          + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode]
          + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray
          + val converter = new ConvertToExpression(inputNames, catalog)
          +
          + conjunctions.asScala.foreach(rex => {
          + rex.accept(converter) match

          { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + }

          + })
          + (convertedExpressions.toArray, unconvertedRexNodes.toArray)
          +
          + case _ => (Array.empty, Array.empty)
          + }
          + }
          +}
          +
          +/**
          + * An RexVisitor to extract all referenced input fields
          + */
          +class InputRefVisitor extends RexVisitorImpl[Unit](true)

          { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +}

          +
          +/**
          + * An RexVisitor to convert RexNode to Expression.
          + *
          + * @param inputNames The input names of the relation node
          + * @param functionCatalog The function catalog
          + */
          +class ConvertToExpression(
          — End diff –

          Rename to `RexNodeToExpressionConverter`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106159389 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala — @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlFunction, SqlPostfixOperator} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util. {Failure, Success, Try} + +object RexProgramExtractor { + + /** + * Extracts the indices of input fields which accessed by the RexProgram. + * + * @param rexProgram The RexProgram to analyze + * @return The indices of accessed input fields + */ + def extractRefInputFields(rexProgram: RexProgram): Array [Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + rexProgram.getProjectList.foreach( + exp => rexProgram.expandLocalRef(exp).accept(visitor)) + + // extract referenced input fields from condition + val condition = rexProgram.getCondition + if (condition != null) { + rexProgram.expandLocalRef(condition).accept(visitor) + } + + visitor.getFields + } + + /** + * Extract condition from RexProgram and convert it into independent CNF expressions. + * + * @param rexProgram The RexProgram to analyze + * @return converted expressions as well as RexNodes which cannot be translated + */ + def extractConjunctiveConditions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): (Array [Expression] , Array [RexNode] ) = { + + rexProgram.getCondition match { + case condition: RexLocalRef => + val expanded = rexProgram.expandLocalRef(condition) + // converts the expanded expression to conjunctive normal form, + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" + val cnf = RexUtil.toCnf(rexBuilder, expanded) + // converts the cnf condition to a list of AND conditions + val conjunctions = RelOptUtil.conjunctions(cnf) + + val convertedExpressions = new mutable.ArrayBuffer [Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer [RexNode] + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray + val converter = new ConvertToExpression(inputNames, catalog) + + conjunctions.asScala.foreach(rex => { + rex.accept(converter) match { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + } + }) + (convertedExpressions.toArray, unconvertedRexNodes.toArray) + + case _ => (Array.empty, Array.empty) + } + } +} + +/** + * An RexVisitor to extract all referenced input fields + */ +class InputRefVisitor extends RexVisitorImpl [Unit] (true) { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +} + +/** + * An RexVisitor to convert RexNode to Expression. + * + * @param inputNames The input names of the relation node + * @param functionCatalog The function catalog + */ +class ConvertToExpression( — End diff – Rename to `RexNodeToExpressionConverter`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106134112

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -0,0 +1,66 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.sources
          +
          +import org.apache.flink.table.expressions.Expression
          +
          +/**
          + * Adds support for filtering push-down to a [[TableSource]].
          + * A [[TableSource]] extending this interface is able to filter records before returning.
          + */
          +trait FilterableTableSource[T] extends TableSource[T] {
          +
          + /**
          + * Indicates whether the filter push down has been applied. Note that even if we don't
          + * actually push down any filters, we should also set this flag to true after the trying.
          + */
          + private var filterPushedDown: Boolean = false
          +
          + /**
          + * Check and pick all predicates this table source can support. The passed in predicates
          + * have been translated in conjunctive form, and table source can only pick those predicates
          + * that it supports.
          + * <p>
          + * After trying to push predicates down, we should return a new [[FilterableTableSource]]
          + * instance which holds all pushed down predicates. Even if we actually pushed nothing down,
          + * it is recommended that we still return a new [[FilterableTableSource]] instance since we will
          + * mark the returned instance as filter push down has been tried. Also we need to return all
          + * unsupported predicates back to the framework to do further filtering.
          + * <p>
          + * We also should note to not changing the form of the predicates passed in. It has been
          + * organized in CNF conjunctive form, and we should only take or leave each element in the
          + * array. Don't try to reorganize the predicates if you are absolutely confident with that.
          + *
          + * @param predicate An array contains conjunctive predicates.
          + * @return A new cloned instance of [[FilterableTableSource]] as well as n array of Expression
          + * which contains all unsupported predicates.
          + */
          + def applyPredicate(predicate: Array[Expression]): (FilterableTableSource[_], Array[Expression])
          +
          + /**
          + * Return the flag to indicate whether filter push down has been tried.
          + */
          + def isFilterPushedDown: Boolean = filterPushedDown
          — End diff –

          I'm not sure about the `isFilterPushedDown` and `setFilterPushDown` methods. They seem to unnecessarily blow up the interface (at least for Java classes that implement the trait).

          Can't we track in `TableSourceScan` if we tried to push a filter down?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106134112 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources + +import org.apache.flink.table.expressions.Expression + +/** + * Adds support for filtering push-down to a [ [TableSource] ]. + * A [ [TableSource] ] extending this interface is able to filter records before returning. + */ +trait FilterableTableSource [T] extends TableSource [T] { + + /** + * Indicates whether the filter push down has been applied. Note that even if we don't + * actually push down any filters, we should also set this flag to true after the trying. + */ + private var filterPushedDown: Boolean = false + + /** + * Check and pick all predicates this table source can support. The passed in predicates + * have been translated in conjunctive form, and table source can only pick those predicates + * that it supports. + * <p> + * After trying to push predicates down, we should return a new [ [FilterableTableSource] ] + * instance which holds all pushed down predicates. Even if we actually pushed nothing down, + * it is recommended that we still return a new [ [FilterableTableSource] ] instance since we will + * mark the returned instance as filter push down has been tried. Also we need to return all + * unsupported predicates back to the framework to do further filtering. + * <p> + * We also should note to not changing the form of the predicates passed in. It has been + * organized in CNF conjunctive form, and we should only take or leave each element in the + * array. Don't try to reorganize the predicates if you are absolutely confident with that. + * + * @param predicate An array contains conjunctive predicates. + * @return A new cloned instance of [ [FilterableTableSource] ] as well as n array of Expression + * which contains all unsupported predicates. + */ + def applyPredicate(predicate: Array [Expression] ): (FilterableTableSource [_] , Array [Expression] ) + + /** + * Return the flag to indicate whether filter push down has been tried. + */ + def isFilterPushedDown: Boolean = filterPushedDown — End diff – I'm not sure about the `isFilterPushedDown` and `setFilterPushDown` methods. They seem to unnecessarily blow up the interface (at least for Java classes that implement the trait). Can't we track in `TableSourceScan` if we tried to push a filter down?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106135176

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -0,0 +1,66 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.sources
          +
          +import org.apache.flink.table.expressions.Expression
          +
          +/**
          + * Adds support for filtering push-down to a [[TableSource]].
          + * A [[TableSource]] extending this interface is able to filter records before returning.
          + */
          +trait FilterableTableSource[T] extends TableSource[T] {
          +
          + /**
          + * Indicates whether the filter push down has been applied. Note that even if we don't
          + * actually push down any filters, we should also set this flag to true after the trying.
          + */
          + private var filterPushedDown: Boolean = false
          +
          + /**
          + * Check and pick all predicates this table source can support. The passed in predicates
          + * have been translated in conjunctive form, and table source can only pick those predicates
          + * that it supports.
          + * <p>
          + * After trying to push predicates down, we should return a new [[FilterableTableSource]]
          + * instance which holds all pushed down predicates. Even if we actually pushed nothing down,
          + * it is recommended that we still return a new [[FilterableTableSource]] instance since we will
          + * mark the returned instance as filter push down has been tried. Also we need to return all
          + * unsupported predicates back to the framework to do further filtering.
          + * <p>
          + * We also should note to not changing the form of the predicates passed in. It has been
          + * organized in CNF conjunctive form, and we should only take or leave each element in the
          + * array. Don't try to reorganize the predicates if you are absolutely confident with that.
          + *
          + * @param predicate An array contains conjunctive predicates.
          + * @return A new cloned instance of [[FilterableTableSource]] as well as n array of Expression
          + * which contains all unsupported predicates.
          + */
          + def applyPredicate(predicate: Array[Expression]): (FilterableTableSource[_], Array[Expression])
          — End diff –

          We should not use Scala classes (`Tuple2`) here to make the interface better compatible with Java.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106135176 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources + +import org.apache.flink.table.expressions.Expression + +/** + * Adds support for filtering push-down to a [ [TableSource] ]. + * A [ [TableSource] ] extending this interface is able to filter records before returning. + */ +trait FilterableTableSource [T] extends TableSource [T] { + + /** + * Indicates whether the filter push down has been applied. Note that even if we don't + * actually push down any filters, we should also set this flag to true after the trying. + */ + private var filterPushedDown: Boolean = false + + /** + * Check and pick all predicates this table source can support. The passed in predicates + * have been translated in conjunctive form, and table source can only pick those predicates + * that it supports. + * <p> + * After trying to push predicates down, we should return a new [ [FilterableTableSource] ] + * instance which holds all pushed down predicates. Even if we actually pushed nothing down, + * it is recommended that we still return a new [ [FilterableTableSource] ] instance since we will + * mark the returned instance as filter push down has been tried. Also we need to return all + * unsupported predicates back to the framework to do further filtering. + * <p> + * We also should note to not changing the form of the predicates passed in. It has been + * organized in CNF conjunctive form, and we should only take or leave each element in the + * array. Don't try to reorganize the predicates if you are absolutely confident with that. + * + * @param predicate An array contains conjunctive predicates. + * @return A new cloned instance of [ [FilterableTableSource] ] as well as n array of Expression + * which contains all unsupported predicates. + */ + def applyPredicate(predicate: Array [Expression] ): (FilterableTableSource [_] , Array [Expression] ) — End diff – We should not use Scala classes (`Tuple2`) here to make the interface better compatible with Java.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106135951

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -0,0 +1,66 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.sources
          +
          +import org.apache.flink.table.expressions.Expression
          +
          +/**
          + * Adds support for filtering push-down to a [[TableSource]].
          + * A [[TableSource]] extending this interface is able to filter records before returning.
          + */
          +trait FilterableTableSource[T] extends TableSource[T] {
          +
          + /**
          + * Indicates whether the filter push down has been applied. Note that even if we don't
          + * actually push down any filters, we should also set this flag to true after the trying.
          + */
          + private var filterPushedDown: Boolean = false
          +
          + /**
          + * Check and pick all predicates this table source can support. The passed in predicates
          + * have been translated in conjunctive form, and table source can only pick those predicates
          + * that it supports.
          + * <p>
          + * After trying to push predicates down, we should return a new [[FilterableTableSource]]
          + * instance which holds all pushed down predicates. Even if we actually pushed nothing down,
          + * it is recommended that we still return a new [[FilterableTableSource]] instance since we will
          + * mark the returned instance as filter push down has been tried. Also we need to return all
          + * unsupported predicates back to the framework to do further filtering.
          + * <p>
          + * We also should note to not changing the form of the predicates passed in. It has been
          + * organized in CNF conjunctive form, and we should only take or leave each element in the
          + * array. Don't try to reorganize the predicates if you are absolutely confident with that.
          + *
          + * @param predicate An array contains conjunctive predicates.
          + * @return A new cloned instance of [[FilterableTableSource]] as well as n array of Expression
          + * which contains all unsupported predicates.
          + */
          + def applyPredicate(predicate: Array[Expression]): (FilterableTableSource[_], Array[Expression])
          — End diff –

          Maybe should design this as:
          ```
          // returns a copy of the table source with pushed down predicates
          applyPredicate(predicate: Array[Expression]): FilterableTableSource[_]

          // returns pushed down predicates
          getPredicates(): Array[Expression]
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106135951 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources + +import org.apache.flink.table.expressions.Expression + +/** + * Adds support for filtering push-down to a [ [TableSource] ]. + * A [ [TableSource] ] extending this interface is able to filter records before returning. + */ +trait FilterableTableSource [T] extends TableSource [T] { + + /** + * Indicates whether the filter push down has been applied. Note that even if we don't + * actually push down any filters, we should also set this flag to true after the trying. + */ + private var filterPushedDown: Boolean = false + + /** + * Check and pick all predicates this table source can support. The passed in predicates + * have been translated in conjunctive form, and table source can only pick those predicates + * that it supports. + * <p> + * After trying to push predicates down, we should return a new [ [FilterableTableSource] ] + * instance which holds all pushed down predicates. Even if we actually pushed nothing down, + * it is recommended that we still return a new [ [FilterableTableSource] ] instance since we will + * mark the returned instance as filter push down has been tried. Also we need to return all + * unsupported predicates back to the framework to do further filtering. + * <p> + * We also should note to not changing the form of the predicates passed in. It has been + * organized in CNF conjunctive form, and we should only take or leave each element in the + * array. Don't try to reorganize the predicates if you are absolutely confident with that. + * + * @param predicate An array contains conjunctive predicates. + * @return A new cloned instance of [ [FilterableTableSource] ] as well as n array of Expression + * which contains all unsupported predicates. + */ + def applyPredicate(predicate: Array [Expression] ): (FilterableTableSource [_] , Array [Expression] ) — End diff – Maybe should design this as: ``` // returns a copy of the table source with pushed down predicates applyPredicate(predicate: Array [Expression] ): FilterableTableSource [_] // returns pushed down predicates getPredicates(): Array [Expression] ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106136200

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.common
          +
          +import org.apache.calcite.plan.RelOptRuleCall
          +import org.apache.calcite.rel.core.Calc
          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.TableSourceScan
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.plan.util.RexProgramExtractor
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +trait PushFilterIntoTableSourceScanRuleBase {
          +
          + private[flink] def pushFilterIntoScan(
          + call: RelOptRuleCall,
          + calc: Calc,
          + scan: TableSourceScan,
          + tableSourceTable: TableSourceTable[_],
          + filterableSource: FilterableTableSource[_],
          + description: String): Unit = {
          +
          + if (filterableSource.isFilterPushedDown)

          { + // The rule can get triggered again due to the transformed "scan => filter" + // sequence created by the earlier execution of this rule when we could not + // push all the conditions into the scan + return + }

          +
          + val program = calc.getProgram
          + val (predicates, unconvertedRexNodes) =
          + RexProgramExtractor.extractConjunctiveConditions(
          + program,
          + call.builder().getRexBuilder,
          + tableSourceTable.tableEnv.getFunctionCatalog)
          + if (predicates.isEmpty)

          { + // no condition can be translated to expression + return + }

          +
          + val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates)
          + // trying to apply filter push down, set the flag to true no matter whether
          + // we actually push any filters down.
          + newTableSource.setFilterPushedDown(true)
          — End diff –

          I think we can set this flag also on `TableSourceScan`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106136200 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.common + +import org.apache.calcite.plan.RelOptRuleCall +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.TableSourceScan +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.util.RexProgramExtractor +import org.apache.flink.table.sources.FilterableTableSource + +trait PushFilterIntoTableSourceScanRuleBase { + + private [flink] def pushFilterIntoScan( + call: RelOptRuleCall, + calc: Calc, + scan: TableSourceScan, + tableSourceTable: TableSourceTable [_] , + filterableSource: FilterableTableSource [_] , + description: String): Unit = { + + if (filterableSource.isFilterPushedDown) { + // The rule can get triggered again due to the transformed "scan => filter" + // sequence created by the earlier execution of this rule when we could not + // push all the conditions into the scan + return + } + + val program = calc.getProgram + val (predicates, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + call.builder().getRexBuilder, + tableSourceTable.tableEnv.getFunctionCatalog) + if (predicates.isEmpty) { + // no condition can be translated to expression + return + } + + val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates) + // trying to apply filter push down, set the flag to true no matter whether + // we actually push any filters down. + newTableSource.setFilterPushedDown(true) — End diff – I think we can set this flag also on `TableSourceScan`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106321885

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/ProjectableTableSource.scala —
          @@ -24,7 +24,7 @@ package org.apache.flink.table.sources
          *

          • @tparam T The return type of the [[ProjectableTableSource]].
            */
            -trait ProjectableTableSource[T] {
            +trait ProjectableTableSource[T] extends TableSource[T] {
              • End diff –

          It's because i want to unify the PushProjectIntoScan rule codes for both batch and stream mode. And once we push down project into table source, we not only should create a new TableScan instance, but also a new TableSource instance. The codes are like:
          ```
          val newTableSource = originTableSource.projectFields(usedFields)
          // create a new scan with the new TableSource instance
          val newScan = scan.copy(scan.getTraitSet, newTableSource)
          ```
          At first the `projectFields` method returned `ProjectableTableSource` which is not a `TableSource`, so i let `ProjectableTableSource` inherit from `TableSource`. But i just noticed we can just let `projectFields` return one `TableSource`, and problem resolved.

          Will change this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106321885 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/ProjectableTableSource.scala — @@ -24,7 +24,7 @@ package org.apache.flink.table.sources * @tparam T The return type of the [ [ProjectableTableSource] ]. */ -trait ProjectableTableSource [T] { +trait ProjectableTableSource [T] extends TableSource [T] { End diff – It's because i want to unify the PushProjectIntoScan rule codes for both batch and stream mode. And once we push down project into table source, we not only should create a new TableScan instance, but also a new TableSource instance. The codes are like: ``` val newTableSource = originTableSource.projectFields(usedFields) // create a new scan with the new TableSource instance val newScan = scan.copy(scan.getTraitSet, newTableSource) ``` At first the `projectFields` method returned `ProjectableTableSource` which is not a `TableSource`, so i let `ProjectableTableSource` inherit from `TableSource`. But i just noticed we can just let `projectFields` return one `TableSource`, and problem resolved. Will change this.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106322568

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala —
          @@ -42,63 +41,40 @@ class DataSetCalc(
          traitSet: RelTraitSet,
          input: RelNode,
          rowRelDataType: RelDataType,

          • private[flink] val calcProgram: RexProgram, // for tests
            + calcProgram: RexProgram,
            ruleDescription: String)
          • extends SingleRel(cluster, traitSet, input)
            + extends Calc(cluster, traitSet, input, calcProgram)
              • End diff –

          This is because i want to unify the PushFilterIntoScan rule's code for both batch and stream mode. During executing the rule, we may need to create a new copy of the DataSetCalc or DataStreamCalc. It make things more easier to let these two classes inherit from `Calc`, and use `Calc.copy` to create a new copied instance.

          I do encountered some problem after i changed the hierarchy, some unit tests failed because of the plan changed. But it's because we don't calculate the cost for Calc right. I added some logic to `CommanCalc.computeSelfCost`, and everything works fine.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106322568 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala — @@ -42,63 +41,40 @@ class DataSetCalc( traitSet: RelTraitSet, input: RelNode, rowRelDataType: RelDataType, private [flink] val calcProgram: RexProgram, // for tests + calcProgram: RexProgram, ruleDescription: String) extends SingleRel(cluster, traitSet, input) + extends Calc(cluster, traitSet, input, calcProgram) End diff – This is because i want to unify the PushFilterIntoScan rule's code for both batch and stream mode. During executing the rule, we may need to create a new copy of the DataSetCalc or DataStreamCalc. It make things more easier to let these two classes inherit from `Calc`, and use `Calc.copy` to create a new copied instance. I do encountered some problem after i changed the hierarchy, some unit tests failed because of the plan changed. But it's because we don't calculate the cost for Calc right. I added some logic to `CommanCalc.computeSelfCost`, and everything works fine.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106324868

          — 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 –

          Make sense to me, will change this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106324868 — 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 – Make sense to me, will change this.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106324883

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala —
          @@ -58,16 +60,24 @@ class BatchTableSourceScan(
          )
          }

          + override def copy(traitSet: RelTraitSet, newTableSource: TableSource[_]): TableSourceScan =

          { + new BatchTableSourceScan( + cluster, + traitSet, + getTable, + newTableSource.asInstanceOf[BatchTableSource[_]] + ) + }

          +
          override def explainTerms(pw: RelWriter): RelWriter = {

          • super.explainTerms(pw)
            + val terms = super.explainTerms(pw)
            .item("fields", TableEnvironment.getFieldNames(tableSource).mkString(", "))
            + tableSource.explainTerms(terms)
              • End diff –

          will change this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106324883 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala — @@ -58,16 +60,24 @@ class BatchTableSourceScan( ) } + override def copy(traitSet: RelTraitSet, newTableSource: TableSource [_] ): TableSourceScan = { + new BatchTableSourceScan( + cluster, + traitSet, + getTable, + newTableSource.asInstanceOf[BatchTableSource[_]] + ) + } + override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) + val terms = super.explainTerms(pw) .item("fields", TableEnvironment.getFieldNames(tableSource).mkString(", ")) + tableSource.explainTerms(terms) End diff – will change this.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106327707

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.common
          +
          +import org.apache.calcite.plan.RelOptRuleCall
          +import org.apache.calcite.rel.core.Calc
          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.TableSourceScan
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.plan.util.RexProgramExtractor
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +trait PushFilterIntoTableSourceScanRuleBase {
          +
          + private[flink] def pushFilterIntoScan(
          + call: RelOptRuleCall,
          + calc: Calc,
          + scan: TableSourceScan,
          + tableSourceTable: TableSourceTable[_],
          + filterableSource: FilterableTableSource[_],
          + description: String): Unit = {
          +
          + if (filterableSource.isFilterPushedDown)

          { + // The rule can get triggered again due to the transformed "scan => filter" + // sequence created by the earlier execution of this rule when we could not + // push all the conditions into the scan + return + }

          +
          + val program = calc.getProgram
          + val (predicates, unconvertedRexNodes) =
          + RexProgramExtractor.extractConjunctiveConditions(
          + program,
          + call.builder().getRexBuilder,
          + tableSourceTable.tableEnv.getFunctionCatalog)
          + if (predicates.isEmpty)

          { + // no condition can be translated to expression + return + }

          +
          + val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates)
          + // trying to apply filter push down, set the flag to true no matter whether
          + // we actually push any filters down.
          + newTableSource.setFilterPushedDown(true)
          +
          + // check whether framework still need to do a filter
          + val relBuilder = call.builder()
          + val remainingCondition = {
          + if (remainingPredicates.nonEmpty || unconvertedRexNodes.nonEmpty)

          { + relBuilder.push(scan) + (remainingPredicates.map(expr => expr.toRexNode(relBuilder)) ++ unconvertedRexNodes) + .reduce((l, r) => relBuilder.and(l, r)) + }

          else

          { + null + }

          + }
          +
          + // check whether we still need a RexProgram. An RexProgram is needed when either
          + // projection or filter exists.
          + val newScan = scan.copy(scan.getTraitSet, newTableSource)
          + val newRexProgram = {
          + if (remainingCondition != null || program.getProjectList.size() > 0) {
          — End diff –

          Thanks for the tips, will change this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106327707 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.common + +import org.apache.calcite.plan.RelOptRuleCall +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.TableSourceScan +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.util.RexProgramExtractor +import org.apache.flink.table.sources.FilterableTableSource + +trait PushFilterIntoTableSourceScanRuleBase { + + private [flink] def pushFilterIntoScan( + call: RelOptRuleCall, + calc: Calc, + scan: TableSourceScan, + tableSourceTable: TableSourceTable [_] , + filterableSource: FilterableTableSource [_] , + description: String): Unit = { + + if (filterableSource.isFilterPushedDown) { + // The rule can get triggered again due to the transformed "scan => filter" + // sequence created by the earlier execution of this rule when we could not + // push all the conditions into the scan + return + } + + val program = calc.getProgram + val (predicates, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + call.builder().getRexBuilder, + tableSourceTable.tableEnv.getFunctionCatalog) + if (predicates.isEmpty) { + // no condition can be translated to expression + return + } + + val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates) + // trying to apply filter push down, set the flag to true no matter whether + // we actually push any filters down. + newTableSource.setFilterPushedDown(true) + + // check whether framework still need to do a filter + val relBuilder = call.builder() + val remainingCondition = { + if (remainingPredicates.nonEmpty || unconvertedRexNodes.nonEmpty) { + relBuilder.push(scan) + (remainingPredicates.map(expr => expr.toRexNode(relBuilder)) ++ unconvertedRexNodes) + .reduce((l, r) => relBuilder.and(l, r)) + } else { + null + } + } + + // check whether we still need a RexProgram. An RexProgram is needed when either + // projection or filter exists. + val newScan = scan.copy(scan.getTraitSet, newTableSource) + val newRexProgram = { + if (remainingCondition != null || program.getProjectList.size() > 0) { — End diff – Thanks for the tips, will change this.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106328751

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala —
          @@ -25,6 +25,7 @@ import org.apache.flink.table.sources.TableSource
          /** Table which defines an external table via a [[TableSource]] */
          class TableSourceTable[T](
          val tableSource: TableSource[T],
          + val tableEnv: TableEnvironment,
          — End diff –

          Yes, you are right, especially that UDF is currently registered as objects but not classes, it's really impossible to let TableSource supporting this. I will remove this filed and only use built-in functions when extracting expression form RexProgram.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106328751 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala — @@ -25,6 +25,7 @@ import org.apache.flink.table.sources.TableSource /** Table which defines an external table via a [ [TableSource] ] */ class TableSourceTable [T] ( val tableSource: TableSource [T] , + val tableEnv: TableEnvironment, — End diff – Yes, you are right, especially that UDF is currently registered as objects but not classes, it's really impossible to let TableSource supporting this. I will remove this filed and only use built-in functions when extracting expression form RexProgram.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106329079

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.common
          +
          +import org.apache.calcite.plan.RelOptRuleCall
          +import org.apache.calcite.rel.core.Calc
          +import org.apache.calcite.rex.RexProgram
          +import org.apache.flink.table.plan.nodes.TableSourceScan
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.plan.util.RexProgramExtractor
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +trait PushFilterIntoTableSourceScanRuleBase {
          +
          + private[flink] def pushFilterIntoScan(
          + call: RelOptRuleCall,
          + calc: Calc,
          + scan: TableSourceScan,
          + tableSourceTable: TableSourceTable[_],
          + filterableSource: FilterableTableSource[_],
          + description: String): Unit = {
          +
          + if (filterableSource.isFilterPushedDown) {
          — End diff –

          Will change this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106329079 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.common + +import org.apache.calcite.plan.RelOptRuleCall +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.TableSourceScan +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.util.RexProgramExtractor +import org.apache.flink.table.sources.FilterableTableSource + +trait PushFilterIntoTableSourceScanRuleBase { + + private [flink] def pushFilterIntoScan( + call: RelOptRuleCall, + calc: Calc, + scan: TableSourceScan, + tableSourceTable: TableSourceTable [_] , + filterableSource: FilterableTableSource [_] , + description: String): Unit = { + + if (filterableSource.isFilterPushedDown) { — End diff – Will change this.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106329099

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala —
          @@ -0,0 +1,184 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlFunction, SqlPostfixOperator}

          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.util.

          {Failure, Success, Try}

          +
          +object RexProgramExtractor {
          +
          + /**
          + * Extracts the indices of input fields which accessed by the RexProgram.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return The indices of accessed input fields
          + */
          + def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
          + val visitor = new InputRefVisitor
          +
          + // extract referenced input fields from projections
          + rexProgram.getProjectList.foreach(
          + exp => rexProgram.expandLocalRef(exp).accept(visitor))
          +
          + // extract referenced input fields from condition
          + val condition = rexProgram.getCondition
          + if (condition != null)

          { + rexProgram.expandLocalRef(condition).accept(visitor) + }

          +
          + visitor.getFields
          + }
          +
          + /**
          + * Extract condition from RexProgram and convert it into independent CNF expressions.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expressions as well as RexNodes which cannot be translated
          + */
          + def extractConjunctiveConditions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = {
          +
          + rexProgram.getCondition match {
          + case condition: RexLocalRef =>
          + val expanded = rexProgram.expandLocalRef(condition)
          + // converts the expanded expression to conjunctive normal form,
          + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)"
          + val cnf = RexUtil.toCnf(rexBuilder, expanded)
          + // converts the cnf condition to a list of AND conditions
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          +
          + val convertedExpressions = new mutable.ArrayBuffer[Expression]
          + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode]
          + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray
          + val converter = new ConvertToExpression(inputNames, catalog)
          +
          + conjunctions.asScala.foreach(rex => {
          + rex.accept(converter) match

          { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + }

          + })
          + (convertedExpressions.toArray, unconvertedRexNodes.toArray)
          +
          + case _ => (Array.empty, Array.empty)
          + }
          + }
          +}
          +
          +/**
          + * An RexVisitor to extract all referenced input fields
          + */
          +class InputRefVisitor extends RexVisitorImpl[Unit](true)

          { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +}

          +
          +/**
          + * An RexVisitor to convert RexNode to Expression.
          + *
          + * @param inputNames The input names of the relation node
          + * @param functionCatalog The function catalog
          + */
          +class ConvertToExpression(
          — End diff –

          Will change.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106329099 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala — @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlFunction, SqlPostfixOperator} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util. {Failure, Success, Try} + +object RexProgramExtractor { + + /** + * Extracts the indices of input fields which accessed by the RexProgram. + * + * @param rexProgram The RexProgram to analyze + * @return The indices of accessed input fields + */ + def extractRefInputFields(rexProgram: RexProgram): Array [Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + rexProgram.getProjectList.foreach( + exp => rexProgram.expandLocalRef(exp).accept(visitor)) + + // extract referenced input fields from condition + val condition = rexProgram.getCondition + if (condition != null) { + rexProgram.expandLocalRef(condition).accept(visitor) + } + + visitor.getFields + } + + /** + * Extract condition from RexProgram and convert it into independent CNF expressions. + * + * @param rexProgram The RexProgram to analyze + * @return converted expressions as well as RexNodes which cannot be translated + */ + def extractConjunctiveConditions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): (Array [Expression] , Array [RexNode] ) = { + + rexProgram.getCondition match { + case condition: RexLocalRef => + val expanded = rexProgram.expandLocalRef(condition) + // converts the expanded expression to conjunctive normal form, + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" + val cnf = RexUtil.toCnf(rexBuilder, expanded) + // converts the cnf condition to a list of AND conditions + val conjunctions = RelOptUtil.conjunctions(cnf) + + val convertedExpressions = new mutable.ArrayBuffer [Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer [RexNode] + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray + val converter = new ConvertToExpression(inputNames, catalog) + + conjunctions.asScala.foreach(rex => { + rex.accept(converter) match { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + } + }) + (convertedExpressions.toArray, unconvertedRexNodes.toArray) + + case _ => (Array.empty, Array.empty) + } + } +} + +/** + * An RexVisitor to extract all referenced input fields + */ +class InputRefVisitor extends RexVisitorImpl [Unit] (true) { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +} + +/** + * An RexVisitor to convert RexNode to Expression. + * + * @param inputNames The input names of the relation node + * @param functionCatalog The function catalog + */ +class ConvertToExpression( — End diff – Will change.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106329219

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala —
          @@ -0,0 +1,55 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.dataSet
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.dataset.

          {BatchTableSourceScan, DataSetCalc}

          +import org.apache.flink.table.plan.rules.common.PushFilterIntoTableSourceScanRuleBase
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataSetCalc],
          + operand(classOf[BatchTableSourceScan], none)),
          + "PushFilterIntoBatchTableSourceScanRule")
          + with PushFilterIntoTableSourceScanRuleBase {
          +
          + override def matches(call: RelOptRuleCall): Boolean = {
          + val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
          + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
          + scan.tableSource match {
          + case _: FilterableTableSource[_] =>
          + calc.getProgram.getCondition != null
          — End diff –

          Will add

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106329219 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushFilterIntoBatchTableSourceScanRule.scala — @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.dataSet + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.dataset. {BatchTableSourceScan, DataSetCalc} +import org.apache.flink.table.plan.rules.common.PushFilterIntoTableSourceScanRuleBase +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoBatchTableSourceScanRule extends RelOptRule( + operand(classOf [DataSetCalc] , + operand(classOf [BatchTableSourceScan] , none)), + "PushFilterIntoBatchTableSourceScanRule") + with PushFilterIntoTableSourceScanRuleBase { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: DataSetCalc = call.rel(0).asInstanceOf [DataSetCalc] + val scan: BatchTableSourceScan = call.rel(1).asInstanceOf [BatchTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource [_] => + calc.getProgram.getCondition != null — End diff – Will add
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106329225

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala —
          @@ -0,0 +1,55 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.datastream
          +
          +import org.apache.calcite.plan.RelOptRule._
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.flink.table.plan.nodes.datastream.

          {DataStreamCalc, StreamTableSourceScan}

          +import org.apache.flink.table.plan.rules.common.PushFilterIntoTableSourceScanRuleBase
          +import org.apache.flink.table.plan.schema.TableSourceTable
          +import org.apache.flink.table.sources.FilterableTableSource
          +
          +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule(
          + operand(classOf[DataStreamCalc],
          + operand(classOf[StreamTableSourceScan], none)),
          + "PushFilterIntoStreamTableSourceScanRule")
          + with PushFilterIntoTableSourceScanRuleBase {
          +
          + override def matches(call: RelOptRuleCall): Boolean = {
          + val calc: DataStreamCalc = call.rel(0).asInstanceOf[DataStreamCalc]
          + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
          + scan.tableSource match {
          + case _: FilterableTableSource[_] =>
          + calc.getProgram.getCondition != null
          — End diff –

          Will add

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106329225 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushFilterIntoStreamTableSourceScanRule.scala — @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.flink.table.plan.nodes.datastream. {DataStreamCalc, StreamTableSourceScan} +import org.apache.flink.table.plan.rules.common.PushFilterIntoTableSourceScanRuleBase +import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.sources.FilterableTableSource + +class PushFilterIntoStreamTableSourceScanRule extends RelOptRule( + operand(classOf [DataStreamCalc] , + operand(classOf [StreamTableSourceScan] , none)), + "PushFilterIntoStreamTableSourceScanRule") + with PushFilterIntoTableSourceScanRuleBase { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: DataStreamCalc = call.rel(0).asInstanceOf [DataStreamCalc] + val scan: StreamTableSourceScan = call.rel(1).asInstanceOf [StreamTableSourceScan] + scan.tableSource match { + case _: FilterableTableSource [_] => + calc.getProgram.getCondition != null — End diff – Will add
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106335660

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala —
          @@ -0,0 +1,184 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlFunction, SqlPostfixOperator}

          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.util.

          {Failure, Success, Try}

          +
          +object RexProgramExtractor {
          +
          + /**
          + * Extracts the indices of input fields which accessed by the RexProgram.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return The indices of accessed input fields
          + */
          + def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
          + val visitor = new InputRefVisitor
          +
          + // extract referenced input fields from projections
          + rexProgram.getProjectList.foreach(
          + exp => rexProgram.expandLocalRef(exp).accept(visitor))
          +
          + // extract referenced input fields from condition
          + val condition = rexProgram.getCondition
          + if (condition != null)

          { + rexProgram.expandLocalRef(condition).accept(visitor) + }

          +
          + visitor.getFields
          + }
          +
          + /**
          + * Extract condition from RexProgram and convert it into independent CNF expressions.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expressions as well as RexNodes which cannot be translated
          + */
          + def extractConjunctiveConditions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = {
          +
          + rexProgram.getCondition match {
          + case condition: RexLocalRef =>
          + val expanded = rexProgram.expandLocalRef(condition)
          + // converts the expanded expression to conjunctive normal form,
          + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)"
          + val cnf = RexUtil.toCnf(rexBuilder, expanded)
          + // converts the cnf condition to a list of AND conditions
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          +
          + val convertedExpressions = new mutable.ArrayBuffer[Expression]
          + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode]
          + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray
          + val converter = new ConvertToExpression(inputNames, catalog)
          +
          + conjunctions.asScala.foreach(rex => {
          + rex.accept(converter) match

          { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + }

          + })
          + (convertedExpressions.toArray, unconvertedRexNodes.toArray)
          +
          + case _ => (Array.empty, Array.empty)
          + }
          + }
          +}
          +
          +/**
          + * An RexVisitor to extract all referenced input fields
          + */
          +class InputRefVisitor extends RexVisitorImpl[Unit](true)

          { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +}

          +
          +/**
          + * An RexVisitor to convert RexNode to Expression.
          + *
          + * @param inputNames The input names of the relation node
          + * @param functionCatalog The function catalog
          + */
          +class ConvertToExpression(
          + inputNames: Array[String],
          + functionCatalog: FunctionCatalog)
          + extends RexVisitor[Option[Expression]] {
          +
          + override def visitInputRef(inputRef: RexInputRef): Option[Expression] =

          { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + }

          +
          + override def visitLocalRef(localRef: RexLocalRef): Option[Expression] = {
          + throw new TableException("Bug: RefLocalRef should have been expanded")
          — End diff –

          fixed

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106335660 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala — @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlFunction, SqlPostfixOperator} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util. {Failure, Success, Try} + +object RexProgramExtractor { + + /** + * Extracts the indices of input fields which accessed by the RexProgram. + * + * @param rexProgram The RexProgram to analyze + * @return The indices of accessed input fields + */ + def extractRefInputFields(rexProgram: RexProgram): Array [Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + rexProgram.getProjectList.foreach( + exp => rexProgram.expandLocalRef(exp).accept(visitor)) + + // extract referenced input fields from condition + val condition = rexProgram.getCondition + if (condition != null) { + rexProgram.expandLocalRef(condition).accept(visitor) + } + + visitor.getFields + } + + /** + * Extract condition from RexProgram and convert it into independent CNF expressions. + * + * @param rexProgram The RexProgram to analyze + * @return converted expressions as well as RexNodes which cannot be translated + */ + def extractConjunctiveConditions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): (Array [Expression] , Array [RexNode] ) = { + + rexProgram.getCondition match { + case condition: RexLocalRef => + val expanded = rexProgram.expandLocalRef(condition) + // converts the expanded expression to conjunctive normal form, + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" + val cnf = RexUtil.toCnf(rexBuilder, expanded) + // converts the cnf condition to a list of AND conditions + val conjunctions = RelOptUtil.conjunctions(cnf) + + val convertedExpressions = new mutable.ArrayBuffer [Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer [RexNode] + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray + val converter = new ConvertToExpression(inputNames, catalog) + + conjunctions.asScala.foreach(rex => { + rex.accept(converter) match { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + } + }) + (convertedExpressions.toArray, unconvertedRexNodes.toArray) + + case _ => (Array.empty, Array.empty) + } + } +} + +/** + * An RexVisitor to extract all referenced input fields + */ +class InputRefVisitor extends RexVisitorImpl [Unit] (true) { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +} + +/** + * An RexVisitor to convert RexNode to Expression. + * + * @param inputNames The input names of the relation node + * @param functionCatalog The function catalog + */ +class ConvertToExpression( + inputNames: Array [String] , + functionCatalog: FunctionCatalog) + extends RexVisitor[Option [Expression] ] { + + override def visitInputRef(inputRef: RexInputRef): Option [Expression] = { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + } + + override def visitLocalRef(localRef: RexLocalRef): Option [Expression] = { + throw new TableException("Bug: RefLocalRef should have been expanded") — End diff – fixed
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106335709

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala —
          @@ -0,0 +1,184 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import org.apache.calcite.plan.RelOptUtil
          +import org.apache.calcite.rex._
          +import org.apache.calcite.sql.

          {SqlFunction, SqlPostfixOperator}

          +import org.apache.flink.table.api.TableException
          +import org.apache.flink.table.calcite.FlinkTypeFactory
          +import org.apache.flink.table.expressions.

          {Expression, Literal, ResolvedFieldReference}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConversions._
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.util.

          {Failure, Success, Try}

          +
          +object RexProgramExtractor {
          +
          + /**
          + * Extracts the indices of input fields which accessed by the RexProgram.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return The indices of accessed input fields
          + */
          + def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
          + val visitor = new InputRefVisitor
          +
          + // extract referenced input fields from projections
          + rexProgram.getProjectList.foreach(
          + exp => rexProgram.expandLocalRef(exp).accept(visitor))
          +
          + // extract referenced input fields from condition
          + val condition = rexProgram.getCondition
          + if (condition != null)

          { + rexProgram.expandLocalRef(condition).accept(visitor) + }

          +
          + visitor.getFields
          + }
          +
          + /**
          + * Extract condition from RexProgram and convert it into independent CNF expressions.
          + *
          + * @param rexProgram The RexProgram to analyze
          + * @return converted expressions as well as RexNodes which cannot be translated
          + */
          + def extractConjunctiveConditions(
          + rexProgram: RexProgram,
          + rexBuilder: RexBuilder,
          + catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = {
          +
          + rexProgram.getCondition match {
          + case condition: RexLocalRef =>
          + val expanded = rexProgram.expandLocalRef(condition)
          + // converts the expanded expression to conjunctive normal form,
          + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)"
          + val cnf = RexUtil.toCnf(rexBuilder, expanded)
          + // converts the cnf condition to a list of AND conditions
          + val conjunctions = RelOptUtil.conjunctions(cnf)
          +
          + val convertedExpressions = new mutable.ArrayBuffer[Expression]
          + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode]
          + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray
          + val converter = new ConvertToExpression(inputNames, catalog)
          +
          + conjunctions.asScala.foreach(rex => {
          + rex.accept(converter) match

          { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + }

          + })
          + (convertedExpressions.toArray, unconvertedRexNodes.toArray)
          +
          + case _ => (Array.empty, Array.empty)
          + }
          + }
          +}
          +
          +/**
          + * An RexVisitor to extract all referenced input fields
          + */
          +class InputRefVisitor extends RexVisitorImpl[Unit](true)

          { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +}

          +
          +/**
          + * An RexVisitor to convert RexNode to Expression.
          + *
          + * @param inputNames The input names of the relation node
          + * @param functionCatalog The function catalog
          + */
          +class ConvertToExpression(
          + inputNames: Array[String],
          + functionCatalog: FunctionCatalog)
          + extends RexVisitor[Option[Expression]] {
          +
          + override def visitInputRef(inputRef: RexInputRef): Option[Expression] =

          { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + }

          +
          + override def visitLocalRef(localRef: RexLocalRef): Option[Expression] =

          { + throw new TableException("Bug: RefLocalRef should have been expanded") + }

          +
          + override def visitLiteral(literal: RexLiteral): Option[Expression] =

          { + Option(Literal(literal.getValue, FlinkTypeFactory.toTypeInfo(literal.getType))) + }

          +
          + override def visitCall(call: RexCall): Option[Expression] = {
          + val operands = call.getOperands.map(
          + operand => operand.accept(this).orNull
          + )
          +
          + // return null if we cannot translate all the operands of the call
          + if (operands.contains(null))

          { + Option.empty + }

          else {
          + call.getOperator match {
          + case function: SqlFunction =>
          + lookupFunction(replace(function.getName), operands)
          + case postfix: SqlPostfixOperator =>
          + lookupFunction(replace(postfix.getName), operands)
          + case operator@_ =>
          + lookupFunction(replace(s"$

          {operator.getKind}

          "), operands)
          + }
          + }
          + }
          +
          + override def visitFieldAccess(fieldAccess: RexFieldAccess): Option[Expression] = Option.empty
          +
          + override def visitCorrelVariable(correlVariable: RexCorrelVariable): Option[Expression] =
          + Option.empty
          +
          + override def visitRangeRef(rangeRef: RexRangeRef): Option[Expression] = Option.empty
          +
          + override def visitSubQuery(subQuery: RexSubQuery): Option[Expression] = Option.empty
          +
          + override def visitDynamicParam(dynamicParam: RexDynamicParam): Option[Expression] = Option.empty
          +
          + override def visitOver(over: RexOver): Option[Expression] = Option.empty
          +
          + private def lookupFunction(name: String, operands: Seq[Expression]): Option[Expression] = {
          + Try(functionCatalog.lookupFunction(name, operands)) match {
          — End diff –

          Yes, i've changed the passed in functionCatalog to an instance which only holds all built-in functions.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106335709 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala — @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ +import org.apache.calcite.sql. {SqlFunction, SqlPostfixOperator} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.expressions. {Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.validate.FunctionCatalog +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util. {Failure, Success, Try} + +object RexProgramExtractor { + + /** + * Extracts the indices of input fields which accessed by the RexProgram. + * + * @param rexProgram The RexProgram to analyze + * @return The indices of accessed input fields + */ + def extractRefInputFields(rexProgram: RexProgram): Array [Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + rexProgram.getProjectList.foreach( + exp => rexProgram.expandLocalRef(exp).accept(visitor)) + + // extract referenced input fields from condition + val condition = rexProgram.getCondition + if (condition != null) { + rexProgram.expandLocalRef(condition).accept(visitor) + } + + visitor.getFields + } + + /** + * Extract condition from RexProgram and convert it into independent CNF expressions. + * + * @param rexProgram The RexProgram to analyze + * @return converted expressions as well as RexNodes which cannot be translated + */ + def extractConjunctiveConditions( + rexProgram: RexProgram, + rexBuilder: RexBuilder, + catalog: FunctionCatalog): (Array [Expression] , Array [RexNode] ) = { + + rexProgram.getCondition match { + case condition: RexLocalRef => + val expanded = rexProgram.expandLocalRef(condition) + // converts the expanded expression to conjunctive normal form, + // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" + val cnf = RexUtil.toCnf(rexBuilder, expanded) + // converts the cnf condition to a list of AND conditions + val conjunctions = RelOptUtil.conjunctions(cnf) + + val convertedExpressions = new mutable.ArrayBuffer [Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer [RexNode] + val inputNames = rexProgram.getInputRowType.getFieldNames.asScala.toArray + val converter = new ConvertToExpression(inputNames, catalog) + + conjunctions.asScala.foreach(rex => { + rex.accept(converter) match { + case Some(expression) => convertedExpressions += expression + case None => unconvertedRexNodes += rex + } + }) + (convertedExpressions.toArray, unconvertedRexNodes.toArray) + + case _ => (Array.empty, Array.empty) + } + } +} + +/** + * An RexVisitor to extract all referenced input fields + */ +class InputRefVisitor extends RexVisitorImpl [Unit] (true) { + + private var fields = mutable.LinkedHashSet[Int]() + + def getFields: Array[Int] = fields.toArray + + override def visitInputRef(inputRef: RexInputRef): Unit = + fields += inputRef.getIndex + + override def visitCall(call: RexCall): Unit = + call.operands.foreach(operand => operand.accept(this)) +} + +/** + * An RexVisitor to convert RexNode to Expression. + * + * @param inputNames The input names of the relation node + * @param functionCatalog The function catalog + */ +class ConvertToExpression( + inputNames: Array [String] , + functionCatalog: FunctionCatalog) + extends RexVisitor[Option [Expression] ] { + + override def visitInputRef(inputRef: RexInputRef): Option [Expression] = { + Preconditions.checkArgument(inputRef.getIndex < inputNames.length) + Option(ResolvedFieldReference( + inputNames(inputRef.getIndex), + FlinkTypeFactory.toTypeInfo(inputRef.getType) + )) + } + + override def visitLocalRef(localRef: RexLocalRef): Option [Expression] = { + throw new TableException("Bug: RefLocalRef should have been expanded") + } + + override def visitLiteral(literal: RexLiteral): Option [Expression] = { + Option(Literal(literal.getValue, FlinkTypeFactory.toTypeInfo(literal.getType))) + } + + override def visitCall(call: RexCall): Option [Expression] = { + val operands = call.getOperands.map( + operand => operand.accept(this).orNull + ) + + // return null if we cannot translate all the operands of the call + if (operands.contains(null)) { + Option.empty + } else { + call.getOperator match { + case function: SqlFunction => + lookupFunction(replace(function.getName), operands) + case postfix: SqlPostfixOperator => + lookupFunction(replace(postfix.getName), operands) + case operator@_ => + lookupFunction(replace(s"$ {operator.getKind} "), operands) + } + } + } + + override def visitFieldAccess(fieldAccess: RexFieldAccess): Option [Expression] = Option.empty + + override def visitCorrelVariable(correlVariable: RexCorrelVariable): Option [Expression] = + Option.empty + + override def visitRangeRef(rangeRef: RexRangeRef): Option [Expression] = Option.empty + + override def visitSubQuery(subQuery: RexSubQuery): Option [Expression] = Option.empty + + override def visitDynamicParam(dynamicParam: RexDynamicParam): Option [Expression] = Option.empty + + override def visitOver(over: RexOver): Option [Expression] = Option.empty + + private def lookupFunction(name: String, operands: Seq [Expression] ): Option [Expression] = { + Try(functionCatalog.lookupFunction(name, operands)) match { — End diff – Yes, i've changed the passed in functionCatalog to an instance which only holds all built-in functions.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106335718

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -0,0 +1,66 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.sources
          +
          +import org.apache.flink.table.expressions.Expression
          +
          +/**
          + * Adds support for filtering push-down to a [[TableSource]].
          + * A [[TableSource]] extending this interface is able to filter records before returning.
          + */
          +trait FilterableTableSource[T] extends TableSource[T] {
          — End diff –

          Changed.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106335718 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources + +import org.apache.flink.table.expressions.Expression + +/** + * Adds support for filtering push-down to a [ [TableSource] ]. + * A [ [TableSource] ] extending this interface is able to filter records before returning. + */ +trait FilterableTableSource [T] extends TableSource [T] { — End diff – Changed.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user KurtYoung commented on the issue:

          https://github.com/apache/flink/pull/3520

          Hi @fhueske @godfreyhe, thanks for the review, i addressed most all your comments.

          @fhueske Except for letting `TableSourceScan` be aware of whether filter has been pushed down. I'm not sure to let the `TableSourceScan` has this kind of information, i'd prefer to let them stay within the all kinds of `TableSource`. One drawback to let `TableSourceScan` has such kind of information is when we do the `TableSourceScan` copy, we need to take care all these information, make sure they also be copied correctly. In the future, if we add more extension to `TableSource` like we can push part of query down, we will face this problem.

          Regarding to the interface of `FilterableTableSource`, i agree with you that the trait containing some logic is not friendly with java extensions. So i removed the default implementation of `isFilterPushedDown`, the inherited class should take care of this method. And regarding the `Tuple2` thing, how about we pass in a mutable java list, and let table source to pick out expression from it and return a copy of table source which contains these pushed down predicates.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on the issue: https://github.com/apache/flink/pull/3520 Hi @fhueske @godfreyhe, thanks for the review, i addressed most all your comments. @fhueske Except for letting `TableSourceScan` be aware of whether filter has been pushed down. I'm not sure to let the `TableSourceScan` has this kind of information, i'd prefer to let them stay within the all kinds of `TableSource`. One drawback to let `TableSourceScan` has such kind of information is when we do the `TableSourceScan` copy, we need to take care all these information, make sure they also be copied correctly. In the future, if we add more extension to `TableSource` like we can push part of query down, we will face this problem. Regarding to the interface of `FilterableTableSource`, i agree with you that the trait containing some logic is not friendly with java extensions. So i removed the default implementation of `isFilterPushedDown`, the inherited class should take care of this method. And regarding the `Tuple2` thing, how about we pass in a mutable java list, and let table source to pick out expression from it and return a copy of table source which contains these pushed down predicates.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106362409

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala —
          @@ -42,63 +41,40 @@ class DataSetCalc(
          traitSet: RelTraitSet,
          input: RelNode,
          rowRelDataType: RelDataType,

          • private[flink] val calcProgram: RexProgram, // for tests
            + calcProgram: RexProgram,
            ruleDescription: String)
          • extends SingleRel(cluster, traitSet, input)
            + extends Calc(cluster, traitSet, input, calcProgram)
              • End diff –

          OK, sounds good to me. I was just curious, because I remembered we had some issues when `DataSetCalc` extended `Calc`. If everything works fine, I'm OK with this change.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106362409 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala — @@ -42,63 +41,40 @@ class DataSetCalc( traitSet: RelTraitSet, input: RelNode, rowRelDataType: RelDataType, private [flink] val calcProgram: RexProgram, // for tests + calcProgram: RexProgram, ruleDescription: String) extends SingleRel(cluster, traitSet, input) + extends Calc(cluster, traitSet, input, calcProgram) End diff – OK, sounds good to me. I was just curious, because I remembered we had some issues when `DataSetCalc` extended `Calc`. If everything works fine, I'm OK with this change.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

          https://github.com/apache/flink/pull/3520

          Thanks for the update @KurtYoung!

          I think you have a good point about not extending `TableSourceScan`. The filter pushdown flag might not be the only one. Let's keep this information in `FilterableTableScan`.

          I'm not so sure about the interface of `applyPredicate(predicates: JList[Expression])`. I think we should not ask users to remove the accepted Expressions from the `predicates` list. How about we add a method `getPredicates(): JList[Expression]` which returns the accepted expressions. This method could also replace `isFilterPushedDown` if we expect `null` if `getPredicates()` was not called and an empty list if it was called but no predicate was accepted. IMO, `getPredicates()` has also the advantage that we can check that the `FilterableTableSource` did not modify the accepted predicates.

          What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3520 Thanks for the update @KurtYoung! I think you have a good point about not extending `TableSourceScan`. The filter pushdown flag might not be the only one. Let's keep this information in `FilterableTableScan`. I'm not so sure about the interface of `applyPredicate(predicates: JList [Expression] )`. I think we should not ask users to remove the accepted Expressions from the `predicates` list. How about we add a method `getPredicates(): JList [Expression] ` which returns the accepted expressions. This method could also replace `isFilterPushedDown` if we expect `null` if `getPredicates()` was not called and an empty list if it was called but no predicate was accepted. IMO, `getPredicates()` has also the advantage that we can check that the `FilterableTableSource` did not modify the accepted predicates. What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user KurtYoung commented on the issue:

          https://github.com/apache/flink/pull/3520

          Hi @fhueske, you just pointed out a question i had when tonycox implementing the first version. Why we are preventing `FilterableTableSource` from modifying the expressions? I think it's totally up to them whether changing the expressions they took or keep them just they were. We have not reason to restrict the `FilterableTableSource`'s behavior if they just do the things right.

          Pass in a java list and told user who extending this to pick and remove the expressions the support is not super nice. But even if user just pick expressions but not remove them, we still get the correct answer.

          What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on the issue: https://github.com/apache/flink/pull/3520 Hi @fhueske, you just pointed out a question i had when tonycox implementing the first version. Why we are preventing `FilterableTableSource` from modifying the expressions? I think it's totally up to them whether changing the expressions they took or keep them just they were. We have not reason to restrict the `FilterableTableSource`'s behavior if they just do the things right. Pass in a java list and told user who extending this to pick and remove the expressions the support is not super nice. But even if user just pick expressions but not remove them, we still get the correct answer. What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

          https://github.com/apache/flink/pull/3520

          Sure, a `FilterableTableSource` can implement its accepted predicates as it likes.
          However, I think we could expect that `getPredicates()` returns the accepted filter expressions as it got them. This makes it easier to check a correct behavior of the `FilterableTableSource`. Internally, the accepted expressions can be decomposed or otherwise changed.

          Similarly, we would need to check that the Expressions which remain in the mutable list are also present in the input list.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3520 Sure, a `FilterableTableSource` can implement its accepted predicates as it likes. However, I think we could expect that `getPredicates()` returns the accepted filter expressions as it got them. This makes it easier to check a correct behavior of the `FilterableTableSource`. Internally, the accepted expressions can be decomposed or otherwise changed. Similarly, we would need to check that the Expressions which remain in the mutable list are also present in the input list.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

          https://github.com/apache/flink/pull/3520

          OK, I thought about this. Let's stick to your mutable list approach. As you said it is safe, i.e., in worst case the table source does not remove expressions and we'll have a bit of overhead due to the unnecessary filters.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3520 OK, I thought about this. Let's stick to your mutable list approach. As you said it is safe, i.e., in worst case the table source does not remove expressions and we'll have a bit of overhead due to the unnecessary filters.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106463685

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala —
          @@ -0,0 +1,347 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import java.math.BigDecimal
          +
          +import org.apache.calcite.rex.

          {RexBuilder, RexProgramBuilder}

          +import org.apache.calcite.sql.SqlPostfixOperator
          +import org.apache.calcite.sql.fun.SqlStdOperatorTable
          +import org.apache.flink.table.expressions.

          {Expression, ExpressionParser}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.junit.Assert.

          {assertArrayEquals, assertEquals}

          +import org.junit.Test
          +
          +import scala.collection.JavaConverters._
          +
          +class RexProgramExtractorTest extends RexProgramTestBase {
          +
          + private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns
          +
          + @Test
          + def testExtractRefInputFields(): Unit =

          { + val usedFields = RexProgramExtractor.extractRefInputFields(buildSimpleRexProgram1()) + assertArrayEquals(usedFields, Array(2, 3, 1)) + }

          +
          + @Test
          + def testExtractSimpleCondition(): Unit =

          { + val builder: RexBuilder = new RexBuilder(typeFactory) + val program = buildSimpleRexProgram2() + + val firstExp = ExpressionParser.parseExpression("id > 6") + val secondExp = ExpressionParser.parseExpression("amount * price < 100") + val expected: Array[Expression] = Array(firstExp, secondExp) + + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + builder, + functionCatalog) + + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractSingleCondition(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + + // a = amount >= id + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)) + builder.addCondition(a) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array(ExpressionParser.parseExpression("amount >= id")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + // ((a AND b) OR c) AND (NOT d) => (a OR c) AND (b OR c) AND (NOT d)
          + @Test
          + def testExtractCnfCondition(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // price + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) + // 100 + val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // a = amount < 100 + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t3)) + // b = id > 100 + val b = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t3)) + // c = price == 100 + val c = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t3)) + // d = amount <= id + val d = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // a AND b + val and = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(a, b).asJava)) + // (a AND b) or c + val or = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.OR, List(and, c).asJava)) + // not d + val not = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT, List(d).asJava)) + + // (a AND b) OR c) AND (NOT d) + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, List(or, not).asJava))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < 100 || price == 100"), + ExpressionParser.parseExpression("id > 100 || price == 100"), + ExpressionParser.parseExpression("!(amount <= id)")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractArithmeticConditions(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + val condition = List( + // amount < id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t1)), + // amount <= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)), + // amount <> id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, t0, t1)), + // amount == id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t0, t1)), + // amount >= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)), + // amount > id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t0, t1)), + // amount + id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.PLUS, t0, t1), t2)), + // amount - id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MINUS, t0, t1), t2)), + // amount * id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t1), t2)), + // amount / id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE, t0, t1), t2)), + // -amount == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.UNARY_MINUS, t0), t2)) + ).asJava + + builder.addCondition(builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, condition))) + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < id"), + ExpressionParser.parseExpression("amount <= id"), + ExpressionParser.parseExpression("amount <> id"), + ExpressionParser.parseExpression("amount == id"), + ExpressionParser.parseExpression("amount >= id"), + ExpressionParser.parseExpression("amount > id"), + ExpressionParser.parseExpression("amount + id == 100"), + ExpressionParser.parseExpression("amount - id == 100"), + ExpressionParser.parseExpression("amount * id == 100"), + ExpressionParser.parseExpression("amount / id == 100"), + ExpressionParser.parseExpression("-amount == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractPostfixConditions(): Unit =

          { + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NULL, "('flag).isNull") + // IS_NOT_NULL will be eliminated since flag is not nullable + // testExtractSinglePostfixCondition(SqlStdOperatorTable.IS_NOT_NULL, "('flag).isNotNull") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_TRUE, "('flag).isTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_TRUE, "('flag).isNotTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_FALSE, "('flag).isFalse") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_FALSE, "('flag).isNotFalse") + }

          +
          + @Test
          + def testExtractConditionWithFunctionCalls(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // sum(amount) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCall(SqlStdOperatorTable.SUM, t0), t2)) + + // min(id) == 100 + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MIN, t1), t2)) + + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("sum(amount) > 100"), + ExpressionParser.parseExpression("min(id) == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractWithUnsupportedConditions(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // unsupported now: amount.cast(BigInteger) + val cast = builder.addExpr(rexBuilder.makeCast(allFieldTypes.get(1), t0)) + + // unsupported now: amount.cast(BigInteger) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCast(allFieldTypes.get(1), t0), t2)) + + // amount <= id + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // contains unsupported condition: (amount.cast(BigInteger) > 100 OR amount <= id) + val condition3 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.OR, condition1, condition2)) + + // only condition2 can be translated + builder.addCondition( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2, condition3)) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount <= id") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(2, unconvertedRexNodes.length) + assertEquals(">(CAST($2):BIGINT NOT NULL, 100)", unconvertedRexNodes(0).toString) + assertEquals("OR(>(CAST($2):BIGINT NOT NULL, 100), <=($2, $1))", + unconvertedRexNodes(1).toString) + }

          +
          + private def testExtractSinglePostfixCondition(
          + fieldIndex: Integer,
          — End diff –

          `fieldIndex` is not used

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106463685 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala — @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import java.math.BigDecimal + +import org.apache.calcite.rex. {RexBuilder, RexProgramBuilder} +import org.apache.calcite.sql.SqlPostfixOperator +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.flink.table.expressions. {Expression, ExpressionParser} +import org.apache.flink.table.validate.FunctionCatalog +import org.junit.Assert. {assertArrayEquals, assertEquals} +import org.junit.Test + +import scala.collection.JavaConverters._ + +class RexProgramExtractorTest extends RexProgramTestBase { + + private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns + + @Test + def testExtractRefInputFields(): Unit = { + val usedFields = RexProgramExtractor.extractRefInputFields(buildSimpleRexProgram1()) + assertArrayEquals(usedFields, Array(2, 3, 1)) + } + + @Test + def testExtractSimpleCondition(): Unit = { + val builder: RexBuilder = new RexBuilder(typeFactory) + val program = buildSimpleRexProgram2() + + val firstExp = ExpressionParser.parseExpression("id > 6") + val secondExp = ExpressionParser.parseExpression("amount * price < 100") + val expected: Array[Expression] = Array(firstExp, secondExp) + + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + builder, + functionCatalog) + + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractSingleCondition(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + + // a = amount >= id + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)) + builder.addCondition(a) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array(ExpressionParser.parseExpression("amount >= id")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + // ((a AND b) OR c) AND (NOT d) => (a OR c) AND (b OR c) AND (NOT d) + @Test + def testExtractCnfCondition(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // price + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) + // 100 + val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // a = amount < 100 + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t3)) + // b = id > 100 + val b = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t3)) + // c = price == 100 + val c = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t3)) + // d = amount <= id + val d = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // a AND b + val and = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(a, b).asJava)) + // (a AND b) or c + val or = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.OR, List(and, c).asJava)) + // not d + val not = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT, List(d).asJava)) + + // (a AND b) OR c) AND (NOT d) + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, List(or, not).asJava))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < 100 || price == 100"), + ExpressionParser.parseExpression("id > 100 || price == 100"), + ExpressionParser.parseExpression("!(amount <= id)")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractArithmeticConditions(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + val condition = List( + // amount < id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t1)), + // amount <= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)), + // amount <> id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, t0, t1)), + // amount == id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t0, t1)), + // amount >= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)), + // amount > id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t0, t1)), + // amount + id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.PLUS, t0, t1), t2)), + // amount - id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MINUS, t0, t1), t2)), + // amount * id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t1), t2)), + // amount / id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE, t0, t1), t2)), + // -amount == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.UNARY_MINUS, t0), t2)) + ).asJava + + builder.addCondition(builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, condition))) + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < id"), + ExpressionParser.parseExpression("amount <= id"), + ExpressionParser.parseExpression("amount <> id"), + ExpressionParser.parseExpression("amount == id"), + ExpressionParser.parseExpression("amount >= id"), + ExpressionParser.parseExpression("amount > id"), + ExpressionParser.parseExpression("amount + id == 100"), + ExpressionParser.parseExpression("amount - id == 100"), + ExpressionParser.parseExpression("amount * id == 100"), + ExpressionParser.parseExpression("amount / id == 100"), + ExpressionParser.parseExpression("-amount == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractPostfixConditions(): Unit = { + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NULL, "('flag).isNull") + // IS_NOT_NULL will be eliminated since flag is not nullable + // testExtractSinglePostfixCondition(SqlStdOperatorTable.IS_NOT_NULL, "('flag).isNotNull") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_TRUE, "('flag).isTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_TRUE, "('flag).isNotTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_FALSE, "('flag).isFalse") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_FALSE, "('flag).isNotFalse") + } + + @Test + def testExtractConditionWithFunctionCalls(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // sum(amount) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCall(SqlStdOperatorTable.SUM, t0), t2)) + + // min(id) == 100 + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MIN, t1), t2)) + + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("sum(amount) > 100"), + ExpressionParser.parseExpression("min(id) == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractWithUnsupportedConditions(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // unsupported now: amount.cast(BigInteger) + val cast = builder.addExpr(rexBuilder.makeCast(allFieldTypes.get(1), t0)) + + // unsupported now: amount.cast(BigInteger) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCast(allFieldTypes.get(1), t0), t2)) + + // amount <= id + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // contains unsupported condition: (amount.cast(BigInteger) > 100 OR amount <= id) + val condition3 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.OR, condition1, condition2)) + + // only condition2 can be translated + builder.addCondition( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2, condition3)) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount <= id") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(2, unconvertedRexNodes.length) + assertEquals(">(CAST($2):BIGINT NOT NULL, 100)", unconvertedRexNodes(0).toString) + assertEquals("OR(>(CAST($2):BIGINT NOT NULL, 100), <=($2, $1))", + unconvertedRexNodes(1).toString) + } + + private def testExtractSinglePostfixCondition( + fieldIndex: Integer, — End diff – `fieldIndex` is not used
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106468487

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala —
          @@ -0,0 +1,142 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.utils
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.java.

          {DataSet, ExecutionEnvironment}

          +import org.apache.flink.streaming.api.datastream.DataStream
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
          +import org.apache.flink.table.api.Types._
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.

          {BatchTableSource, FilterableTableSource, StreamTableSource, TableSource}

          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.tools.nsc.interpreter.JList
          +
          +/**
          + * This source can only handle simple comparision with field "amount".
          + * Supports ">, <, >=, <=, =, <>" with an integer.
          + */
          +class TestFilterableTableSource(
          + val recordNum: Int = 33)
          + extends BatchTableSource[Row]
          + with StreamTableSource[Row]
          + with FilterableTableSource[Row] {
          +
          + var filterPushedDown: Boolean = false
          +
          + val fieldNames: Array[String] = Array("name", "id", "amount", "price")
          +
          + val fieldTypes: Array[TypeInformation[_]] = Array(STRING, LONG, INT, DOUBLE)
          +
          + // all predicates with filed "amount"
          + private var filterPredicates = new mutable.ArrayBuffer[Expression]
          +
          + // all comparing values for field "amount"
          + private val filterValues = new mutable.ArrayBuffer[Int]
          +
          + override def getDataSet(execEnv: ExecutionEnvironment): DataSet[Row] =

          { + execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) + }
          +
          + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { + execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) + }

          +
          + override def explainSource(): String = {
          + if (filterPredicates.nonEmpty) {
          + s"filter=[$

          {filterPredicates.reduce((l, r) => And(l, r)).toString}

          ]"
          + } else

          { + "" + }

          + }
          +
          + override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes, fieldNames)
          +
          + override def applyPredicate(predicates: JList[Expression]): TableSource[Row] = {
          + val newSource = new TestFilterableTableSource(recordNum)
          + newSource.filterPushedDown = true
          +
          + val iterator = predicates.iterator()
          + while (iterator.hasNext) {
          + iterator.next() match {
          + case expr: BinaryComparison =>
          + (expr.left, expr.right) match

          { + case (f: ResolvedFieldReference, v: Literal) if f.name.equals("amount") => + newSource.filterPredicates += expr + newSource.filterValues += v.value.asInstanceOf[Number].intValue() + iterator.remove() + case (_, _) => + }

          + }
          + }
          +
          + newSource
          + }
          +
          + override def isFilterPushedDown: Boolean = filterPushedDown
          +
          + private def generateDynamicCollection(): Seq[Row] = {
          + Preconditions.checkArgument(filterPredicates.length == filterValues.length)
          +
          + for

          { + cnt <- 0 until recordNum + if shouldCreateRow(cnt) + }

          yield {
          + val row = new Row(fieldNames.length)
          — End diff –

          can be simplified to
          ```
          val row = Row.of(
          s"Record_$cnt",
          cnt.toLong.asInstanceOf[Object],
          cnt.toInt.asInstanceOf[Object],
          cnt.toDouble.asInstanceOf[Object])
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106468487 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala — @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.utils + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.java. {DataSet, ExecutionEnvironment} +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.table.api.Types._ +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources. {BatchTableSource, FilterableTableSource, StreamTableSource, TableSource} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.tools.nsc.interpreter.JList + +/** + * This source can only handle simple comparision with field "amount". + * Supports ">, <, >=, <=, =, <>" with an integer. + */ +class TestFilterableTableSource( + val recordNum: Int = 33) + extends BatchTableSource [Row] + with StreamTableSource [Row] + with FilterableTableSource [Row] { + + var filterPushedDown: Boolean = false + + val fieldNames: Array [String] = Array("name", "id", "amount", "price") + + val fieldTypes: Array[TypeInformation [_] ] = Array(STRING, LONG, INT, DOUBLE) + + // all predicates with filed "amount" + private var filterPredicates = new mutable.ArrayBuffer [Expression] + + // all comparing values for field "amount" + private val filterValues = new mutable.ArrayBuffer [Int] + + override def getDataSet(execEnv: ExecutionEnvironment): DataSet [Row] = { + execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) + } + + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream [Row] = { + execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) + } + + override def explainSource(): String = { + if (filterPredicates.nonEmpty) { + s"filter=[$ {filterPredicates.reduce((l, r) => And(l, r)).toString} ]" + } else { + "" + } + } + + override def getReturnType: TypeInformation [Row] = new RowTypeInfo(fieldTypes, fieldNames) + + override def applyPredicate(predicates: JList [Expression] ): TableSource [Row] = { + val newSource = new TestFilterableTableSource(recordNum) + newSource.filterPushedDown = true + + val iterator = predicates.iterator() + while (iterator.hasNext) { + iterator.next() match { + case expr: BinaryComparison => + (expr.left, expr.right) match { + case (f: ResolvedFieldReference, v: Literal) if f.name.equals("amount") => + newSource.filterPredicates += expr + newSource.filterValues += v.value.asInstanceOf[Number].intValue() + iterator.remove() + case (_, _) => + } + } + } + + newSource + } + + override def isFilterPushedDown: Boolean = filterPushedDown + + private def generateDynamicCollection(): Seq [Row] = { + Preconditions.checkArgument(filterPredicates.length == filterValues.length) + + for { + cnt <- 0 until recordNum + if shouldCreateRow(cnt) + } yield { + val row = new Row(fieldNames.length) — End diff – can be simplified to ``` val row = Row.of( s"Record_$cnt", cnt.toLong.asInstanceOf [Object] , cnt.toInt.asInstanceOf [Object] , cnt.toDouble.asInstanceOf [Object] ) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106452031

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableSourceTest.scala —
          @@ -0,0 +1,382 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table
          +
          +import org.apache.flink.table.api.Types
          +import org.apache.flink.table.api.scala._
          +import org.apache.flink.table.sources.

          {CsvTableSource, TableSource}

          +import org.apache.flink.table.utils.TableTestUtil._
          +import org.apache.flink.table.expressions.utils._
          +import org.apache.flink.table.utils.

          {CommonTestData, TableTestBase, TestFilterableTableSource}

          +import org.junit.

          {Assert, Test}

          +
          +class TableSourceTest extends TableTestBase {
          +
          + private val projectedFields: Array[String] = Array("last", "id", "score")
          + private val noCalcFields: Array[String] = Array("id", "score", "first")
          +
          + // batch plan
          +
          + @Test
          + def testBatchProjectableSourceScanPlanTableApi(): Unit =

          { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('last.upperCase(), 'id.floor(), 'score * 2) + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode(tableName, projectedFields), + term("select", "UPPER(last) AS _c0", "FLOOR(id) AS _c1", "*(score, 2) AS _c2") + ) + + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchProjectableSourceScanPlanSQL(): Unit =

          { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + + util.tEnv.registerTableSource(tableName, tableSource) + + val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName" + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode(tableName, projectedFields), + term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2") + ) + + util.verifySql(sqlQuery, expected) + }

          +
          + @Test
          + def testBatchProjectableSourceScanNoIdentityCalc(): Unit =

          { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('id, 'score, 'first) + + val expected = batchSourceTableNode(tableName, noCalcFields) + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterableWithoutPushDown(): Unit =

          { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("price * 2 < 32") + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode( + tableName, + Array("name", "id", "amount", "price")), + term("select", "price", "id", "amount"), + term("where", "<(*(price, 2), 32)") + ) + + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterablePartialPushDown(): Unit =

          { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .where("amount > 2 && price * 2 < 32") + .select('price, 'name.lowerCase(), 'amount) + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2"), + term("select", "price", "LOWER(name) AS _c1", "amount"), + term("where", "<(*(price, 2), 32)") + ) + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterableFullyPushedDown(): Unit =

          { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && amount < 32") + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2 && 'amount < 32"), + term("select", "price", "id", "amount") + ) + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterableWithUnconvertedExpression(): Unit =

          { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && (amount < 32 || amount.cast(LONG) > 10)") // cast can not be converted + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2"), + term("select", "price", "id", "amount"), + term("where", "OR(<(amount, 32), >(CAST(amount), 10))") + ) + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterableWithUDF(): Unit = {
          + val (tableSource, tableName) = filterableTableSource
          + val util = batchTestUtil()
          + val tEnv = util.tEnv
          +
          + tEnv.registerTableSource(tableName, tableSource)
          + tEnv.registerFunction("func0", Func0)
          +
          + val result = tEnv
          + .scan(tableName)
          + .select('price, 'id, 'amount)
          + .where("amount > 2 && func0(amount) < 32")
          +
          + // wo don't fail during the optimization
          — End diff –

          `wo` -> `we`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106452031 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableSourceTest.scala — @@ -0,0 +1,382 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table + +import org.apache.flink.table.api.Types +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.sources. {CsvTableSource, TableSource} +import org.apache.flink.table.utils.TableTestUtil._ +import org.apache.flink.table.expressions.utils._ +import org.apache.flink.table.utils. {CommonTestData, TableTestBase, TestFilterableTableSource} +import org.junit. {Assert, Test} + +class TableSourceTest extends TableTestBase { + + private val projectedFields: Array [String] = Array("last", "id", "score") + private val noCalcFields: Array [String] = Array("id", "score", "first") + + // batch plan + + @Test + def testBatchProjectableSourceScanPlanTableApi(): Unit = { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('last.upperCase(), 'id.floor(), 'score * 2) + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode(tableName, projectedFields), + term("select", "UPPER(last) AS _c0", "FLOOR(id) AS _c1", "*(score, 2) AS _c2") + ) + + util.verifyTable(result, expected) + } + + @Test + def testBatchProjectableSourceScanPlanSQL(): Unit = { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + + util.tEnv.registerTableSource(tableName, tableSource) + + val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName" + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode(tableName, projectedFields), + term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2") + ) + + util.verifySql(sqlQuery, expected) + } + + @Test + def testBatchProjectableSourceScanNoIdentityCalc(): Unit = { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('id, 'score, 'first) + + val expected = batchSourceTableNode(tableName, noCalcFields) + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterableWithoutPushDown(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("price * 2 < 32") + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode( + tableName, + Array("name", "id", "amount", "price")), + term("select", "price", "id", "amount"), + term("where", "<(*(price, 2), 32)") + ) + + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterablePartialPushDown(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .where("amount > 2 && price * 2 < 32") + .select('price, 'name.lowerCase(), 'amount) + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2"), + term("select", "price", "LOWER(name) AS _c1", "amount"), + term("where", "<(*(price, 2), 32)") + ) + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterableFullyPushedDown(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && amount < 32") + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2 && 'amount < 32"), + term("select", "price", "id", "amount") + ) + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterableWithUnconvertedExpression(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && (amount < 32 || amount.cast(LONG) > 10)") // cast can not be converted + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2"), + term("select", "price", "id", "amount"), + term("where", "OR(<(amount, 32), >(CAST(amount), 10))") + ) + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterableWithUDF(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + tEnv.registerFunction("func0", Func0) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && func0(amount) < 32") + + // wo don't fail during the optimization — End diff – `wo` -> `we`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106460094

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramTestBase.scala —
          @@ -16,106 +16,96 @@

          • limitations under the License.
            */

          -package org.apache.flink.table.plan.rules.util
          +package org.apache.flink.table.plan.util

          import java.math.BigDecimal
          +import java.util

          import org.apache.calcite.adapter.java.JavaTypeFactory
          import org.apache.calcite.jdbc.JavaTypeFactoryImpl
          import org.apache.calcite.rel.`type`.

          {RelDataType, RelDataTypeSystem}

          import org.apache.calcite.rex.

          {RexBuilder, RexProgram, RexProgramBuilder}

          -import org.apache.calcite.sql.`type`.SqlTypeName.

          {BIGINT, DOUBLE, INTEGER, VARCHAR}

          +import org.apache.calcite.sql.`type`.SqlTypeName.

          {BIGINT, DOUBLE, INTEGER, VARCHAR, BOOLEAN}

          import org.apache.calcite.sql.fun.SqlStdOperatorTable
          -import org.apache.flink.table.plan.rules.util.RexProgramProjectExtractor._
          -import org.junit.Assert.

          {assertArrayEquals, assertTrue}

          -import org.junit.

          {Before, Test}

          import scala.collection.JavaConverters._
          +import scala.collection.mutable

          -/**

          • * This class is responsible for testing RexProgramProjectExtractor.
          • */
            -class RexProgramProjectExtractorTest {
          • private var typeFactory: JavaTypeFactory = _
          • private var rexBuilder: RexBuilder = _
          • private var allFieldTypes: Seq[RelDataType] = _
          • private val allFieldNames = List("name", "id", "amount", "price")
            -
          • @Before
          • def setUp(): Unit = { - typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT) - rexBuilder = new RexBuilder(typeFactory) - allFieldTypes = List(VARCHAR, BIGINT, INTEGER, DOUBLE).map(typeFactory.createSqlType(_)) - }

            +abstract class RexProgramTestBase {

              • End diff –

          good idea to make this an abstract class

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106460094 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramTestBase.scala — @@ -16,106 +16,96 @@ limitations under the License. */ -package org.apache.flink.table.plan.rules.util +package org.apache.flink.table.plan.util import java.math.BigDecimal +import java.util import org.apache.calcite.adapter.java.JavaTypeFactory import org.apache.calcite.jdbc.JavaTypeFactoryImpl import org.apache.calcite.rel.`type`. {RelDataType, RelDataTypeSystem} import org.apache.calcite.rex. {RexBuilder, RexProgram, RexProgramBuilder} -import org.apache.calcite.sql.`type`.SqlTypeName. {BIGINT, DOUBLE, INTEGER, VARCHAR} +import org.apache.calcite.sql.`type`.SqlTypeName. {BIGINT, DOUBLE, INTEGER, VARCHAR, BOOLEAN} import org.apache.calcite.sql.fun.SqlStdOperatorTable -import org.apache.flink.table.plan.rules.util.RexProgramProjectExtractor._ -import org.junit.Assert. {assertArrayEquals, assertTrue} -import org.junit. {Before, Test} import scala.collection.JavaConverters._ +import scala.collection.mutable -/** * This class is responsible for testing RexProgramProjectExtractor. */ -class RexProgramProjectExtractorTest { private var typeFactory: JavaTypeFactory = _ private var rexBuilder: RexBuilder = _ private var allFieldTypes: Seq [RelDataType] = _ private val allFieldNames = List("name", "id", "amount", "price") - @Before def setUp(): Unit = { - typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT) - rexBuilder = new RexBuilder(typeFactory) - allFieldTypes = List(VARCHAR, BIGINT, INTEGER, DOUBLE).map(typeFactory.createSqlType(_)) - } +abstract class RexProgramTestBase { End diff – good idea to make this an abstract class
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106445598

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -36,36 +43,34 @@ trait PushFilterIntoTableSourceScanRuleBase {
          filterableSource: FilterableTableSource[_],
          description: String): Unit = {

          • if (filterableSource.isFilterPushedDown) { - // The rule can get triggered again due to the transformed "scan => filter" - // sequence created by the earlier execution of this rule when we could not - // push all the conditions into the scan - return - }

            + Preconditions.checkArgument(!filterableSource.isFilterPushedDown)

          val program = calc.getProgram
          + val functionCatalog = FunctionCatalog.withBuiltIns
          val (predicates, unconvertedRexNodes) =
          RexProgramExtractor.extractConjunctiveConditions(
          program,
          call.builder().getRexBuilder,

          • tableSourceTable.tableEnv.getFunctionCatalog)
            + functionCatalog)
            if (predicates.isEmpty) { // no condition can be translated to expression return }
          • val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates)
          • // trying to apply filter push down, set the flag to true no matter whether
          • // we actually push any filters down.
          • newTableSource.setFilterPushedDown(true)
            + val remainingPredicates = new util.LinkedList[Expression]()
            + predicates.foreach(e => remainingPredicates.add(e))
            +
            + val newTableSource = filterableSource.applyPredicate(remainingPredicates)
              • End diff –

          Add a check that `remainingPredicates` is a subset of `predicates`?
          The table source should not touch those predicates that it cannot evaluate or add new predicates.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106445598 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -36,36 +43,34 @@ trait PushFilterIntoTableSourceScanRuleBase { filterableSource: FilterableTableSource [_] , description: String): Unit = { if (filterableSource.isFilterPushedDown) { - // The rule can get triggered again due to the transformed "scan => filter" - // sequence created by the earlier execution of this rule when we could not - // push all the conditions into the scan - return - } + Preconditions.checkArgument(!filterableSource.isFilterPushedDown) val program = calc.getProgram + val functionCatalog = FunctionCatalog.withBuiltIns val (predicates, unconvertedRexNodes) = RexProgramExtractor.extractConjunctiveConditions( program, call.builder().getRexBuilder, tableSourceTable.tableEnv.getFunctionCatalog) + functionCatalog) if (predicates.isEmpty) { // no condition can be translated to expression return } val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates) // trying to apply filter push down, set the flag to true no matter whether // we actually push any filters down. newTableSource.setFilterPushedDown(true) + val remainingPredicates = new util.LinkedList [Expression] () + predicates.foreach(e => remainingPredicates.add(e)) + + val newTableSource = filterableSource.applyPredicate(remainingPredicates) End diff – Add a check that `remainingPredicates` is a subset of `predicates`? The table source should not touch those predicates that it cannot evaluate or add new predicates.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106467291

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala —
          @@ -0,0 +1,142 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.utils
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.java.

          {DataSet, ExecutionEnvironment}

          +import org.apache.flink.streaming.api.datastream.DataStream
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
          +import org.apache.flink.table.api.Types._
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.

          {BatchTableSource, FilterableTableSource, StreamTableSource, TableSource}

          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.tools.nsc.interpreter.JList
          +
          +/**
          + * This source can only handle simple comparision with field "amount".
          + * Supports ">, <, >=, <=, =, <>" with an integer.
          + */
          +class TestFilterableTableSource(
          + val recordNum: Int = 33)
          + extends BatchTableSource[Row]
          + with StreamTableSource[Row]
          + with FilterableTableSource[Row] {
          +
          + var filterPushedDown: Boolean = false
          +
          + val fieldNames: Array[String] = Array("name", "id", "amount", "price")
          +
          + val fieldTypes: Array[TypeInformation[_]] = Array(STRING, LONG, INT, DOUBLE)
          +
          + // all predicates with filed "amount"
          — End diff –

          filed -> field

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106467291 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala — @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.utils + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.java. {DataSet, ExecutionEnvironment} +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.table.api.Types._ +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources. {BatchTableSource, FilterableTableSource, StreamTableSource, TableSource} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.tools.nsc.interpreter.JList + +/** + * This source can only handle simple comparision with field "amount". + * Supports ">, <, >=, <=, =, <>" with an integer. + */ +class TestFilterableTableSource( + val recordNum: Int = 33) + extends BatchTableSource [Row] + with StreamTableSource [Row] + with FilterableTableSource [Row] { + + var filterPushedDown: Boolean = false + + val fieldNames: Array [String] = Array("name", "id", "amount", "price") + + val fieldTypes: Array[TypeInformation [_] ] = Array(STRING, LONG, INT, DOUBLE) + + // all predicates with filed "amount" — End diff – filed -> field
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106459926

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramTestBase.scala —
          @@ -16,106 +16,96 @@

          • limitations under the License.
            */

          -package org.apache.flink.table.plan.rules.util
          +package org.apache.flink.table.plan.util

          import java.math.BigDecimal
          +import java.util

          import org.apache.calcite.adapter.java.JavaTypeFactory
          import org.apache.calcite.jdbc.JavaTypeFactoryImpl
          import org.apache.calcite.rel.`type`.

          {RelDataType, RelDataTypeSystem}

          import org.apache.calcite.rex.

          {RexBuilder, RexProgram, RexProgramBuilder}

          -import org.apache.calcite.sql.`type`.SqlTypeName.

          {BIGINT, DOUBLE, INTEGER, VARCHAR}

          +import org.apache.calcite.sql.`type`.SqlTypeName.

          {BIGINT, DOUBLE, INTEGER, VARCHAR, BOOLEAN}

          import org.apache.calcite.sql.fun.SqlStdOperatorTable
          -import org.apache.flink.table.plan.rules.util.RexProgramProjectExtractor._
          -import org.junit.Assert.

          {assertArrayEquals, assertTrue}

          -import org.junit.

          {Before, Test}

          import scala.collection.JavaConverters._
          +import scala.collection.mutable

          -/**

          • * This class is responsible for testing RexProgramProjectExtractor.
          • */
            -class RexProgramProjectExtractorTest {
          • private var typeFactory: JavaTypeFactory = _
          • private var rexBuilder: RexBuilder = _
          • private var allFieldTypes: Seq[RelDataType] = _
          • private val allFieldNames = List("name", "id", "amount", "price")
            -
          • @Before
          • def setUp(): Unit = { - typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT) - rexBuilder = new RexBuilder(typeFactory) - allFieldTypes = List(VARCHAR, BIGINT, INTEGER, DOUBLE).map(typeFactory.createSqlType(_)) - }

            +abstract class RexProgramTestBase {

          • @Test
          • def testExtractRefInputFields(): Unit = { - val usedFields = extractRefInputFields(buildRexProgram()) - assertArrayEquals(usedFields, Array(2, 3, 1)) - }

            + val typeFactory: JavaTypeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT)
            +
            + val allFieldNames: util.List[String] = List("name", "id", "amount", "price", "flag").asJava
            +
            + val allFieldTypes: util.List[RelDataType] =
            + List(VARCHAR, BIGINT, INTEGER, DOUBLE, BOOLEAN).map(typeFactory.createSqlType).asJava
            +
            + var rexBuilder: RexBuilder = new RexBuilder(typeFactory)

          • @Test
          • def testRewriteRexProgram(): Unit = {
          • val originRexProgram = buildRexProgram()
          • assertTrue(extractExprStrList(originRexProgram).sameElements(Array(
          • "$0",
          • "$1",
          • "$2",
          • "$3",
          • "*($t2, $t3)",
          • "100",
          • "<($t4, $t5)",
          • "6",
          • ">($t1, $t7)",
          • "AND($t6, $t8)")))
          • // use amount, id, price fields to create a new RexProgram
          • val usedFields = Array(2, 3, 1)
          • val types = usedFields.map(allFieldTypes(_)).toList.asJava
          • val names = usedFields.map(allFieldNames(_)).toList.asJava
          • val inputRowType = typeFactory.createStructType(types, names)
          • val newRexProgram = rewriteRexProgram(originRexProgram, inputRowType, usedFields, rexBuilder)
          • assertTrue(extractExprStrList(newRexProgram).sameElements(Array(
          • "$0",
          • "$1",
          • "$2",
          • "*($t0, $t1)",
          • "100",
          • "<($t3, $t4)",
          • "6",
          • ">($t2, $t6)",
          • "AND($t5, $t7)")))
            + /**
            + * extract all expression string list from input RexProgram expression lists
            + *
            + * @param rexProgram input RexProgram instance to analyze
            + * @return all expression string list of input RexProgram expression lists
            + */
            + protected def extractExprStrList(rexProgram: RexProgram): mutable.Buffer[String] = { + rexProgram.getExprList.asScala.map(_.toString) }
          • private def buildRexProgram(): RexProgram = {
          • val types = allFieldTypes.asJava
          • val names = allFieldNames.asJava
          • val inputRowType = typeFactory.createStructType(types, names)
            + // select amount, amount * price as total where amount * price < 100 and id > 6
            + protected def buildSimpleRexProgram1(): RexProgram = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) val builder = new RexProgramBuilder(inputRowType, rexBuilder) - val t0 = rexBuilder.makeInputRef(types.get(2), 2) - val t1 = rexBuilder.makeInputRef(types.get(1), 1) - val t2 = rexBuilder.makeInputRef(types.get(3), 3) + + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) val t3 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t2)) val t4 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) val t5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(6L)) - // project: amount, amount * price + + // project: amount, amount * price as total builder.addProject(t0, "amount") builder.addProject(t3, "total") + // condition: amount * price < 100 and id > 6 val t6 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t3, t4)) val t7 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t5)) val t8 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(t6, t7).asJava)) builder.addCondition(t8) + builder.getProgram }
          • /**
          • * extract all expression string list from input RexProgram expression lists
          • *
          • * @param rexProgram input RexProgram instance to analyze
          • * @return all expression string list of input RexProgram expression lists
          • */
          • private def extractExprStrList(rexProgram: RexProgram) = {
          • rexProgram.getExprList.asScala.map(_.toString)
            + // select amount, amount * price as total
            + // where (amount * price < 100) AND (id > 6)
            + protected def buildSimpleRexProgram2(): RexProgram = {
              • End diff –

          How is this different from `buildSimpleRexProgram1()`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106459926 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramTestBase.scala — @@ -16,106 +16,96 @@ limitations under the License. */ -package org.apache.flink.table.plan.rules.util +package org.apache.flink.table.plan.util import java.math.BigDecimal +import java.util import org.apache.calcite.adapter.java.JavaTypeFactory import org.apache.calcite.jdbc.JavaTypeFactoryImpl import org.apache.calcite.rel.`type`. {RelDataType, RelDataTypeSystem} import org.apache.calcite.rex. {RexBuilder, RexProgram, RexProgramBuilder} -import org.apache.calcite.sql.`type`.SqlTypeName. {BIGINT, DOUBLE, INTEGER, VARCHAR} +import org.apache.calcite.sql.`type`.SqlTypeName. {BIGINT, DOUBLE, INTEGER, VARCHAR, BOOLEAN} import org.apache.calcite.sql.fun.SqlStdOperatorTable -import org.apache.flink.table.plan.rules.util.RexProgramProjectExtractor._ -import org.junit.Assert. {assertArrayEquals, assertTrue} -import org.junit. {Before, Test} import scala.collection.JavaConverters._ +import scala.collection.mutable -/** * This class is responsible for testing RexProgramProjectExtractor. */ -class RexProgramProjectExtractorTest { private var typeFactory: JavaTypeFactory = _ private var rexBuilder: RexBuilder = _ private var allFieldTypes: Seq [RelDataType] = _ private val allFieldNames = List("name", "id", "amount", "price") - @Before def setUp(): Unit = { - typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT) - rexBuilder = new RexBuilder(typeFactory) - allFieldTypes = List(VARCHAR, BIGINT, INTEGER, DOUBLE).map(typeFactory.createSqlType(_)) - } +abstract class RexProgramTestBase { @Test def testExtractRefInputFields(): Unit = { - val usedFields = extractRefInputFields(buildRexProgram()) - assertArrayEquals(usedFields, Array(2, 3, 1)) - } + val typeFactory: JavaTypeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT) + + val allFieldNames: util.List [String] = List("name", "id", "amount", "price", "flag").asJava + + val allFieldTypes: util.List [RelDataType] = + List(VARCHAR, BIGINT, INTEGER, DOUBLE, BOOLEAN).map(typeFactory.createSqlType).asJava + + var rexBuilder: RexBuilder = new RexBuilder(typeFactory) @Test def testRewriteRexProgram(): Unit = { val originRexProgram = buildRexProgram() assertTrue(extractExprStrList(originRexProgram).sameElements(Array( "$0", "$1", "$2", "$3", "*($t2, $t3)", "100", "<($t4, $t5)", "6", ">($t1, $t7)", "AND($t6, $t8)"))) // use amount, id, price fields to create a new RexProgram val usedFields = Array(2, 3, 1) val types = usedFields.map(allFieldTypes(_)).toList.asJava val names = usedFields.map(allFieldNames(_)).toList.asJava val inputRowType = typeFactory.createStructType(types, names) val newRexProgram = rewriteRexProgram(originRexProgram, inputRowType, usedFields, rexBuilder) assertTrue(extractExprStrList(newRexProgram).sameElements(Array( "$0", "$1", "$2", "*($t0, $t1)", "100", "<($t3, $t4)", "6", ">($t2, $t6)", "AND($t5, $t7)"))) + /** + * extract all expression string list from input RexProgram expression lists + * + * @param rexProgram input RexProgram instance to analyze + * @return all expression string list of input RexProgram expression lists + */ + protected def extractExprStrList(rexProgram: RexProgram): mutable.Buffer [String] = { + rexProgram.getExprList.asScala.map(_.toString) } private def buildRexProgram(): RexProgram = { val types = allFieldTypes.asJava val names = allFieldNames.asJava val inputRowType = typeFactory.createStructType(types, names) + // select amount, amount * price as total where amount * price < 100 and id > 6 + protected def buildSimpleRexProgram1(): RexProgram = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) val builder = new RexProgramBuilder(inputRowType, rexBuilder) - val t0 = rexBuilder.makeInputRef(types.get(2), 2) - val t1 = rexBuilder.makeInputRef(types.get(1), 1) - val t2 = rexBuilder.makeInputRef(types.get(3), 3) + + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) val t3 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t2)) val t4 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) val t5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(6L)) - // project: amount, amount * price + + // project: amount, amount * price as total builder.addProject(t0, "amount") builder.addProject(t3, "total") + // condition: amount * price < 100 and id > 6 val t6 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t3, t4)) val t7 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t5)) val t8 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(t6, t7).asJava)) builder.addCondition(t8) + builder.getProgram } /** * extract all expression string list from input RexProgram expression lists * * @param rexProgram input RexProgram instance to analyze * @return all expression string list of input RexProgram expression lists */ private def extractExprStrList(rexProgram: RexProgram) = { rexProgram.getExprList.asScala.map(_.toString) + // select amount, amount * price as total + // where (amount * price < 100) AND (id > 6) + protected def buildSimpleRexProgram2(): RexProgram = { End diff – How is this different from `buildSimpleRexProgram1()`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106447875

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala —
          @@ -18,20 +18,20 @@

          package org.apache.flink.table.utils

          -import org.apache.calcite.rel.RelWriter
          import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.api.java.typeutils.RowTypeInfo
          import org.apache.flink.api.java.

          {DataSet, ExecutionEnvironment}

          import org.apache.flink.streaming.api.datastream.DataStream
          import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
          import org.apache.flink.table.api.Types._
          import org.apache.flink.table.expressions._
          -import org.apache.flink.table.sources.

          {BatchTableSource, FilterableTableSource, StreamTableSource}

          +import org.apache.flink.table.sources.

          {BatchTableSource, FilterableTableSource, StreamTableSource, TableSource}

          import org.apache.flink.types.Row
          import org.apache.flink.util.Preconditions

          import scala.collection.JavaConverters._
          import scala.collection.mutable
          +import scala.tools.nsc.interpreter.JList
          — End diff –

          change to `import java.util.

          {List => JList}

          `?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106447875 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala — @@ -18,20 +18,20 @@ package org.apache.flink.table.utils -import org.apache.calcite.rel.RelWriter import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java. {DataSet, ExecutionEnvironment} import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api.Types._ import org.apache.flink.table.expressions._ -import org.apache.flink.table.sources. {BatchTableSource, FilterableTableSource, StreamTableSource} +import org.apache.flink.table.sources. {BatchTableSource, FilterableTableSource, StreamTableSource, TableSource} import org.apache.flink.types.Row import org.apache.flink.util.Preconditions import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.tools.nsc.interpreter.JList — End diff – change to `import java.util. {List => JList} `?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106446820

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -20,47 +20,40 @@ package org.apache.flink.table.sources

          import org.apache.flink.table.expressions.Expression

          +import scala.tools.nsc.interpreter.JList
          — End diff –

          Can we make this `import java.util.

          {List => JList}

          `?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106446820 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -20,47 +20,40 @@ package org.apache.flink.table.sources import org.apache.flink.table.expressions.Expression +import scala.tools.nsc.interpreter.JList — End diff – Can we make this `import java.util. {List => JList} `?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106464689

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala —
          @@ -0,0 +1,347 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import java.math.BigDecimal
          +
          +import org.apache.calcite.rex.

          {RexBuilder, RexProgramBuilder}

          +import org.apache.calcite.sql.SqlPostfixOperator
          +import org.apache.calcite.sql.fun.SqlStdOperatorTable
          +import org.apache.flink.table.expressions.

          {Expression, ExpressionParser}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.junit.Assert.

          {assertArrayEquals, assertEquals}

          +import org.junit.Test
          +
          +import scala.collection.JavaConverters._
          +
          +class RexProgramExtractorTest extends RexProgramTestBase {
          +
          + private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns
          +
          + @Test
          + def testExtractRefInputFields(): Unit =

          { + val usedFields = RexProgramExtractor.extractRefInputFields(buildSimpleRexProgram1()) + assertArrayEquals(usedFields, Array(2, 3, 1)) + }

          +
          + @Test
          + def testExtractSimpleCondition(): Unit =

          { + val builder: RexBuilder = new RexBuilder(typeFactory) + val program = buildSimpleRexProgram2() + + val firstExp = ExpressionParser.parseExpression("id > 6") + val secondExp = ExpressionParser.parseExpression("amount * price < 100") + val expected: Array[Expression] = Array(firstExp, secondExp) + + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + builder, + functionCatalog) + + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractSingleCondition(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + + // a = amount >= id + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)) + builder.addCondition(a) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array(ExpressionParser.parseExpression("amount >= id")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + // ((a AND b) OR c) AND (NOT d) => (a OR c) AND (b OR c) AND (NOT d)
          + @Test
          + def testExtractCnfCondition(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // price + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) + // 100 + val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // a = amount < 100 + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t3)) + // b = id > 100 + val b = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t3)) + // c = price == 100 + val c = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t3)) + // d = amount <= id + val d = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // a AND b + val and = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(a, b).asJava)) + // (a AND b) or c + val or = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.OR, List(and, c).asJava)) + // not d + val not = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT, List(d).asJava)) + + // (a AND b) OR c) AND (NOT d) + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, List(or, not).asJava))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < 100 || price == 100"), + ExpressionParser.parseExpression("id > 100 || price == 100"), + ExpressionParser.parseExpression("!(amount <= id)")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractArithmeticConditions(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + val condition = List( + // amount < id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t1)), + // amount <= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)), + // amount <> id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, t0, t1)), + // amount == id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t0, t1)), + // amount >= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)), + // amount > id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t0, t1)), + // amount + id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.PLUS, t0, t1), t2)), + // amount - id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MINUS, t0, t1), t2)), + // amount * id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t1), t2)), + // amount / id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE, t0, t1), t2)), + // -amount == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.UNARY_MINUS, t0), t2)) + ).asJava + + builder.addCondition(builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, condition))) + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < id"), + ExpressionParser.parseExpression("amount <= id"), + ExpressionParser.parseExpression("amount <> id"), + ExpressionParser.parseExpression("amount == id"), + ExpressionParser.parseExpression("amount >= id"), + ExpressionParser.parseExpression("amount > id"), + ExpressionParser.parseExpression("amount + id == 100"), + ExpressionParser.parseExpression("amount - id == 100"), + ExpressionParser.parseExpression("amount * id == 100"), + ExpressionParser.parseExpression("amount / id == 100"), + ExpressionParser.parseExpression("-amount == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractPostfixConditions(): Unit =

          { + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NULL, "('flag).isNull") + // IS_NOT_NULL will be eliminated since flag is not nullable + // testExtractSinglePostfixCondition(SqlStdOperatorTable.IS_NOT_NULL, "('flag).isNotNull") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_TRUE, "('flag).isTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_TRUE, "('flag).isNotTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_FALSE, "('flag).isFalse") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_FALSE, "('flag).isNotFalse") + }

          +
          + @Test
          + def testExtractConditionWithFunctionCalls(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // sum(amount) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCall(SqlStdOperatorTable.SUM, t0), t2)) + + // min(id) == 100 + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MIN, t1), t2)) + + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("sum(amount) > 100"), + ExpressionParser.parseExpression("min(id) == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractWithUnsupportedConditions(): Unit = {
          + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames)
          + val builder = new RexProgramBuilder(inputRowType, rexBuilder)
          +
          + // amount
          + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2)
          + // id
          + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1)
          + // 100
          + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L))
          +
          + // unsupported now: amount.cast(BigInteger)
          + val cast = builder.addExpr(rexBuilder.makeCast(allFieldTypes.get(1), t0))
          — End diff –

          `cast` is not used

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106464689 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala — @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import java.math.BigDecimal + +import org.apache.calcite.rex. {RexBuilder, RexProgramBuilder} +import org.apache.calcite.sql.SqlPostfixOperator +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.flink.table.expressions. {Expression, ExpressionParser} +import org.apache.flink.table.validate.FunctionCatalog +import org.junit.Assert. {assertArrayEquals, assertEquals} +import org.junit.Test + +import scala.collection.JavaConverters._ + +class RexProgramExtractorTest extends RexProgramTestBase { + + private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns + + @Test + def testExtractRefInputFields(): Unit = { + val usedFields = RexProgramExtractor.extractRefInputFields(buildSimpleRexProgram1()) + assertArrayEquals(usedFields, Array(2, 3, 1)) + } + + @Test + def testExtractSimpleCondition(): Unit = { + val builder: RexBuilder = new RexBuilder(typeFactory) + val program = buildSimpleRexProgram2() + + val firstExp = ExpressionParser.parseExpression("id > 6") + val secondExp = ExpressionParser.parseExpression("amount * price < 100") + val expected: Array[Expression] = Array(firstExp, secondExp) + + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + builder, + functionCatalog) + + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractSingleCondition(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + + // a = amount >= id + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)) + builder.addCondition(a) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array(ExpressionParser.parseExpression("amount >= id")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + // ((a AND b) OR c) AND (NOT d) => (a OR c) AND (b OR c) AND (NOT d) + @Test + def testExtractCnfCondition(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // price + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) + // 100 + val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // a = amount < 100 + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t3)) + // b = id > 100 + val b = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t3)) + // c = price == 100 + val c = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t3)) + // d = amount <= id + val d = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // a AND b + val and = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(a, b).asJava)) + // (a AND b) or c + val or = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.OR, List(and, c).asJava)) + // not d + val not = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT, List(d).asJava)) + + // (a AND b) OR c) AND (NOT d) + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, List(or, not).asJava))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < 100 || price == 100"), + ExpressionParser.parseExpression("id > 100 || price == 100"), + ExpressionParser.parseExpression("!(amount <= id)")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractArithmeticConditions(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + val condition = List( + // amount < id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t1)), + // amount <= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)), + // amount <> id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, t0, t1)), + // amount == id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t0, t1)), + // amount >= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)), + // amount > id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t0, t1)), + // amount + id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.PLUS, t0, t1), t2)), + // amount - id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MINUS, t0, t1), t2)), + // amount * id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t1), t2)), + // amount / id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE, t0, t1), t2)), + // -amount == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.UNARY_MINUS, t0), t2)) + ).asJava + + builder.addCondition(builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, condition))) + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < id"), + ExpressionParser.parseExpression("amount <= id"), + ExpressionParser.parseExpression("amount <> id"), + ExpressionParser.parseExpression("amount == id"), + ExpressionParser.parseExpression("amount >= id"), + ExpressionParser.parseExpression("amount > id"), + ExpressionParser.parseExpression("amount + id == 100"), + ExpressionParser.parseExpression("amount - id == 100"), + ExpressionParser.parseExpression("amount * id == 100"), + ExpressionParser.parseExpression("amount / id == 100"), + ExpressionParser.parseExpression("-amount == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractPostfixConditions(): Unit = { + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NULL, "('flag).isNull") + // IS_NOT_NULL will be eliminated since flag is not nullable + // testExtractSinglePostfixCondition(SqlStdOperatorTable.IS_NOT_NULL, "('flag).isNotNull") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_TRUE, "('flag).isTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_TRUE, "('flag).isNotTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_FALSE, "('flag).isFalse") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_FALSE, "('flag).isNotFalse") + } + + @Test + def testExtractConditionWithFunctionCalls(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // sum(amount) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCall(SqlStdOperatorTable.SUM, t0), t2)) + + // min(id) == 100 + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MIN, t1), t2)) + + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("sum(amount) > 100"), + ExpressionParser.parseExpression("min(id) == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractWithUnsupportedConditions(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // unsupported now: amount.cast(BigInteger) + val cast = builder.addExpr(rexBuilder.makeCast(allFieldTypes.get(1), t0)) — End diff – `cast` is not used
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106451995

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableSourceTest.scala —
          @@ -0,0 +1,382 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table
          +
          +import org.apache.flink.table.api.Types
          +import org.apache.flink.table.api.scala._
          +import org.apache.flink.table.sources.

          {CsvTableSource, TableSource}

          +import org.apache.flink.table.utils.TableTestUtil._
          +import org.apache.flink.table.expressions.utils._
          +import org.apache.flink.table.utils.

          {CommonTestData, TableTestBase, TestFilterableTableSource}

          +import org.junit.

          {Assert, Test}

          +
          +class TableSourceTest extends TableTestBase {
          +
          + private val projectedFields: Array[String] = Array("last", "id", "score")
          + private val noCalcFields: Array[String] = Array("id", "score", "first")
          +
          + // batch plan
          +
          + @Test
          + def testBatchProjectableSourceScanPlanTableApi(): Unit =

          { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('last.upperCase(), 'id.floor(), 'score * 2) + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode(tableName, projectedFields), + term("select", "UPPER(last) AS _c0", "FLOOR(id) AS _c1", "*(score, 2) AS _c2") + ) + + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchProjectableSourceScanPlanSQL(): Unit =

          { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + + util.tEnv.registerTableSource(tableName, tableSource) + + val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName" + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode(tableName, projectedFields), + term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2") + ) + + util.verifySql(sqlQuery, expected) + }

          +
          + @Test
          + def testBatchProjectableSourceScanNoIdentityCalc(): Unit =

          { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('id, 'score, 'first) + + val expected = batchSourceTableNode(tableName, noCalcFields) + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterableWithoutPushDown(): Unit =

          { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("price * 2 < 32") + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode( + tableName, + Array("name", "id", "amount", "price")), + term("select", "price", "id", "amount"), + term("where", "<(*(price, 2), 32)") + ) + + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterablePartialPushDown(): Unit =

          { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .where("amount > 2 && price * 2 < 32") + .select('price, 'name.lowerCase(), 'amount) + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2"), + term("select", "price", "LOWER(name) AS _c1", "amount"), + term("where", "<(*(price, 2), 32)") + ) + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterableFullyPushedDown(): Unit =

          { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && amount < 32") + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2 && 'amount < 32"), + term("select", "price", "id", "amount") + ) + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterableWithUnconvertedExpression(): Unit =

          { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && (amount < 32 || amount.cast(LONG) > 10)") // cast can not be converted + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2"), + term("select", "price", "id", "amount"), + term("where", "OR(<(amount, 32), >(CAST(amount), 10))") + ) + util.verifyTable(result, expected) + }

          +
          + @Test
          + def testBatchFilterableWithUDF(): Unit = {
          + val (tableSource, tableName) = filterableTableSource
          + val util = batchTestUtil()
          + val tEnv = util.tEnv
          +
          + tEnv.registerTableSource(tableName, tableSource)
          + tEnv.registerFunction("func0", Func0)
          +
          + val result = tEnv
          + .scan(tableName)
          + .select('price, 'id, 'amount)
          + .where("amount > 2 && func0(amount) < 32")
          +
          + // wo don't fail during the optimization
          + tEnv.optimize(result.getRelNode)
          — End diff –

          Can we check the result as well?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106451995 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableSourceTest.scala — @@ -0,0 +1,382 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table + +import org.apache.flink.table.api.Types +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.sources. {CsvTableSource, TableSource} +import org.apache.flink.table.utils.TableTestUtil._ +import org.apache.flink.table.expressions.utils._ +import org.apache.flink.table.utils. {CommonTestData, TableTestBase, TestFilterableTableSource} +import org.junit. {Assert, Test} + +class TableSourceTest extends TableTestBase { + + private val projectedFields: Array [String] = Array("last", "id", "score") + private val noCalcFields: Array [String] = Array("id", "score", "first") + + // batch plan + + @Test + def testBatchProjectableSourceScanPlanTableApi(): Unit = { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('last.upperCase(), 'id.floor(), 'score * 2) + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode(tableName, projectedFields), + term("select", "UPPER(last) AS _c0", "FLOOR(id) AS _c1", "*(score, 2) AS _c2") + ) + + util.verifyTable(result, expected) + } + + @Test + def testBatchProjectableSourceScanPlanSQL(): Unit = { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + + util.tEnv.registerTableSource(tableName, tableSource) + + val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName" + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode(tableName, projectedFields), + term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2") + ) + + util.verifySql(sqlQuery, expected) + } + + @Test + def testBatchProjectableSourceScanNoIdentityCalc(): Unit = { + val (tableSource, tableName) = csvTable + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('id, 'score, 'first) + + val expected = batchSourceTableNode(tableName, noCalcFields) + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterableWithoutPushDown(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("price * 2 < 32") + + val expected = unaryNode( + "DataSetCalc", + batchSourceTableNode( + tableName, + Array("name", "id", "amount", "price")), + term("select", "price", "id", "amount"), + term("where", "<(*(price, 2), 32)") + ) + + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterablePartialPushDown(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .where("amount > 2 && price * 2 < 32") + .select('price, 'name.lowerCase(), 'amount) + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2"), + term("select", "price", "LOWER(name) AS _c1", "amount"), + term("where", "<(*(price, 2), 32)") + ) + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterableFullyPushedDown(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && amount < 32") + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2 && 'amount < 32"), + term("select", "price", "id", "amount") + ) + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterableWithUnconvertedExpression(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && (amount < 32 || amount.cast(LONG) > 10)") // cast can not be converted + + val expected = unaryNode( + "DataSetCalc", + batchFilterableSourceTableNode( + tableName, + Array("name", "id", "amount", "price"), + "'amount > 2"), + term("select", "price", "id", "amount"), + term("where", "OR(<(amount, 32), >(CAST(amount), 10))") + ) + util.verifyTable(result, expected) + } + + @Test + def testBatchFilterableWithUDF(): Unit = { + val (tableSource, tableName) = filterableTableSource + val util = batchTestUtil() + val tEnv = util.tEnv + + tEnv.registerTableSource(tableName, tableSource) + tEnv.registerFunction("func0", Func0) + + val result = tEnv + .scan(tableName) + .select('price, 'id, 'amount) + .where("amount > 2 && func0(amount) < 32") + + // wo don't fail during the optimization + tEnv.optimize(result.getRelNode) — End diff – Can we check the result as well?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106565668

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala —
          @@ -36,36 +43,34 @@ trait PushFilterIntoTableSourceScanRuleBase {
          filterableSource: FilterableTableSource[_],
          description: String): Unit = {

          • if (filterableSource.isFilterPushedDown) { - // The rule can get triggered again due to the transformed "scan => filter" - // sequence created by the earlier execution of this rule when we could not - // push all the conditions into the scan - return - }

            + Preconditions.checkArgument(!filterableSource.isFilterPushedDown)

          val program = calc.getProgram
          + val functionCatalog = FunctionCatalog.withBuiltIns
          val (predicates, unconvertedRexNodes) =
          RexProgramExtractor.extractConjunctiveConditions(
          program,
          call.builder().getRexBuilder,

          • tableSourceTable.tableEnv.getFunctionCatalog)
            + functionCatalog)
            if (predicates.isEmpty) { // no condition can be translated to expression return }
          • val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates)
          • // trying to apply filter push down, set the flag to true no matter whether
          • // we actually push any filters down.
          • newTableSource.setFilterPushedDown(true)
            + val remainingPredicates = new util.LinkedList[Expression]()
            + predicates.foreach(e => remainingPredicates.add(e))
            +
            + val newTableSource = filterableSource.applyPredicate(remainingPredicates)
              • End diff –

          I think we don't have to restrict this. If the user for some reason indeed want to change the predicates which returns back and executed by framework, we should allow them to do so.

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106565668 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushFilterIntoTableSourceScanRuleBase.scala — @@ -36,36 +43,34 @@ trait PushFilterIntoTableSourceScanRuleBase { filterableSource: FilterableTableSource [_] , description: String): Unit = { if (filterableSource.isFilterPushedDown) { - // The rule can get triggered again due to the transformed "scan => filter" - // sequence created by the earlier execution of this rule when we could not - // push all the conditions into the scan - return - } + Preconditions.checkArgument(!filterableSource.isFilterPushedDown) val program = calc.getProgram + val functionCatalog = FunctionCatalog.withBuiltIns val (predicates, unconvertedRexNodes) = RexProgramExtractor.extractConjunctiveConditions( program, call.builder().getRexBuilder, tableSourceTable.tableEnv.getFunctionCatalog) + functionCatalog) if (predicates.isEmpty) { // no condition can be translated to expression return } val (newTableSource, remainingPredicates) = filterableSource.applyPredicate(predicates) // trying to apply filter push down, set the flag to true no matter whether // we actually push any filters down. newTableSource.setFilterPushedDown(true) + val remainingPredicates = new util.LinkedList [Expression] () + predicates.foreach(e => remainingPredicates.add(e)) + + val newTableSource = filterableSource.applyPredicate(remainingPredicates) End diff – I think we don't have to restrict this. If the user for some reason indeed want to change the predicates which returns back and executed by framework, we should allow them to do so.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106565688

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala —
          @@ -20,47 +20,40 @@ package org.apache.flink.table.sources

          import org.apache.flink.table.expressions.Expression

          +import scala.tools.nsc.interpreter.JList
          — End diff –

          sure

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106565688 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/FilterableTableSource.scala — @@ -20,47 +20,40 @@ package org.apache.flink.table.sources import org.apache.flink.table.expressions.Expression +import scala.tools.nsc.interpreter.JList — End diff – sure
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106566042

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala —
          @@ -0,0 +1,142 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.utils
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.java.

          {DataSet, ExecutionEnvironment}

          +import org.apache.flink.streaming.api.datastream.DataStream
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
          +import org.apache.flink.table.api.Types._
          +import org.apache.flink.table.expressions._
          +import org.apache.flink.table.sources.

          {BatchTableSource, FilterableTableSource, StreamTableSource, TableSource}

          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Preconditions
          +
          +import scala.collection.JavaConverters._
          +import scala.collection.mutable
          +import scala.tools.nsc.interpreter.JList
          +
          +/**
          + * This source can only handle simple comparision with field "amount".
          + * Supports ">, <, >=, <=, =, <>" with an integer.
          + */
          +class TestFilterableTableSource(
          + val recordNum: Int = 33)
          + extends BatchTableSource[Row]
          + with StreamTableSource[Row]
          + with FilterableTableSource[Row] {
          +
          + var filterPushedDown: Boolean = false
          +
          + val fieldNames: Array[String] = Array("name", "id", "amount", "price")
          +
          + val fieldTypes: Array[TypeInformation[_]] = Array(STRING, LONG, INT, DOUBLE)
          +
          + // all predicates with filed "amount"
          + private var filterPredicates = new mutable.ArrayBuffer[Expression]
          +
          + // all comparing values for field "amount"
          + private val filterValues = new mutable.ArrayBuffer[Int]
          +
          + override def getDataSet(execEnv: ExecutionEnvironment): DataSet[Row] =

          { + execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) + }
          +
          + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { + execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) + }

          +
          + override def explainSource(): String = {
          + if (filterPredicates.nonEmpty) {
          + s"filter=[$

          {filterPredicates.reduce((l, r) => And(l, r)).toString}

          ]"
          + } else

          { + "" + }

          + }
          +
          + override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes, fieldNames)
          +
          + override def applyPredicate(predicates: JList[Expression]): TableSource[Row] = {
          + val newSource = new TestFilterableTableSource(recordNum)
          + newSource.filterPushedDown = true
          +
          + val iterator = predicates.iterator()
          + while (iterator.hasNext) {
          + iterator.next() match {
          + case expr: BinaryComparison =>
          + (expr.left, expr.right) match

          { + case (f: ResolvedFieldReference, v: Literal) if f.name.equals("amount") => + newSource.filterPredicates += expr + newSource.filterValues += v.value.asInstanceOf[Number].intValue() + iterator.remove() + case (_, _) => + }

          + }
          + }
          +
          + newSource
          + }
          +
          + override def isFilterPushedDown: Boolean = filterPushedDown
          +
          + private def generateDynamicCollection(): Seq[Row] = {
          + Preconditions.checkArgument(filterPredicates.length == filterValues.length)
          +
          + for

          { + cnt <- 0 until recordNum + if shouldCreateRow(cnt) + }

          yield {
          + val row = new Row(fieldNames.length)
          — End diff –

          Thanks for the tips!

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106566042 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala — @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.utils + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.java. {DataSet, ExecutionEnvironment} +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.table.api.Types._ +import org.apache.flink.table.expressions._ +import org.apache.flink.table.sources. {BatchTableSource, FilterableTableSource, StreamTableSource, TableSource} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.tools.nsc.interpreter.JList + +/** + * This source can only handle simple comparision with field "amount". + * Supports ">, <, >=, <=, =, <>" with an integer. + */ +class TestFilterableTableSource( + val recordNum: Int = 33) + extends BatchTableSource [Row] + with StreamTableSource [Row] + with FilterableTableSource [Row] { + + var filterPushedDown: Boolean = false + + val fieldNames: Array [String] = Array("name", "id", "amount", "price") + + val fieldTypes: Array[TypeInformation [_] ] = Array(STRING, LONG, INT, DOUBLE) + + // all predicates with filed "amount" + private var filterPredicates = new mutable.ArrayBuffer [Expression] + + // all comparing values for field "amount" + private val filterValues = new mutable.ArrayBuffer [Int] + + override def getDataSet(execEnv: ExecutionEnvironment): DataSet [Row] = { + execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) + } + + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream [Row] = { + execEnv.fromCollection[Row](generateDynamicCollection().asJava, getReturnType) + } + + override def explainSource(): String = { + if (filterPredicates.nonEmpty) { + s"filter=[$ {filterPredicates.reduce((l, r) => And(l, r)).toString} ]" + } else { + "" + } + } + + override def getReturnType: TypeInformation [Row] = new RowTypeInfo(fieldTypes, fieldNames) + + override def applyPredicate(predicates: JList [Expression] ): TableSource [Row] = { + val newSource = new TestFilterableTableSource(recordNum) + newSource.filterPushedDown = true + + val iterator = predicates.iterator() + while (iterator.hasNext) { + iterator.next() match { + case expr: BinaryComparison => + (expr.left, expr.right) match { + case (f: ResolvedFieldReference, v: Literal) if f.name.equals("amount") => + newSource.filterPredicates += expr + newSource.filterValues += v.value.asInstanceOf[Number].intValue() + iterator.remove() + case (_, _) => + } + } + } + + newSource + } + + override def isFilterPushedDown: Boolean = filterPushedDown + + private def generateDynamicCollection(): Seq [Row] = { + Preconditions.checkArgument(filterPredicates.length == filterValues.length) + + for { + cnt <- 0 until recordNum + if shouldCreateRow(cnt) + } yield { + val row = new Row(fieldNames.length) — End diff – Thanks for the tips!
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106566599

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala —
          @@ -0,0 +1,347 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.util
          +
          +import java.math.BigDecimal
          +
          +import org.apache.calcite.rex.

          {RexBuilder, RexProgramBuilder}

          +import org.apache.calcite.sql.SqlPostfixOperator
          +import org.apache.calcite.sql.fun.SqlStdOperatorTable
          +import org.apache.flink.table.expressions.

          {Expression, ExpressionParser}

          +import org.apache.flink.table.validate.FunctionCatalog
          +import org.junit.Assert.

          {assertArrayEquals, assertEquals}

          +import org.junit.Test
          +
          +import scala.collection.JavaConverters._
          +
          +class RexProgramExtractorTest extends RexProgramTestBase {
          +
          + private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns
          +
          + @Test
          + def testExtractRefInputFields(): Unit =

          { + val usedFields = RexProgramExtractor.extractRefInputFields(buildSimpleRexProgram1()) + assertArrayEquals(usedFields, Array(2, 3, 1)) + }

          +
          + @Test
          + def testExtractSimpleCondition(): Unit =

          { + val builder: RexBuilder = new RexBuilder(typeFactory) + val program = buildSimpleRexProgram2() + + val firstExp = ExpressionParser.parseExpression("id > 6") + val secondExp = ExpressionParser.parseExpression("amount * price < 100") + val expected: Array[Expression] = Array(firstExp, secondExp) + + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + builder, + functionCatalog) + + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractSingleCondition(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + + // a = amount >= id + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)) + builder.addCondition(a) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array(ExpressionParser.parseExpression("amount >= id")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + // ((a AND b) OR c) AND (NOT d) => (a OR c) AND (b OR c) AND (NOT d)
          + @Test
          + def testExtractCnfCondition(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // price + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) + // 100 + val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // a = amount < 100 + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t3)) + // b = id > 100 + val b = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t3)) + // c = price == 100 + val c = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t3)) + // d = amount <= id + val d = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // a AND b + val and = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(a, b).asJava)) + // (a AND b) or c + val or = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.OR, List(and, c).asJava)) + // not d + val not = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT, List(d).asJava)) + + // (a AND b) OR c) AND (NOT d) + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, List(or, not).asJava))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < 100 || price == 100"), + ExpressionParser.parseExpression("id > 100 || price == 100"), + ExpressionParser.parseExpression("!(amount <= id)")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractArithmeticConditions(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + val condition = List( + // amount < id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t1)), + // amount <= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)), + // amount <> id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, t0, t1)), + // amount == id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t0, t1)), + // amount >= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)), + // amount > id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t0, t1)), + // amount + id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.PLUS, t0, t1), t2)), + // amount - id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MINUS, t0, t1), t2)), + // amount * id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t1), t2)), + // amount / id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE, t0, t1), t2)), + // -amount == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.UNARY_MINUS, t0), t2)) + ).asJava + + builder.addCondition(builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, condition))) + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < id"), + ExpressionParser.parseExpression("amount <= id"), + ExpressionParser.parseExpression("amount <> id"), + ExpressionParser.parseExpression("amount == id"), + ExpressionParser.parseExpression("amount >= id"), + ExpressionParser.parseExpression("amount > id"), + ExpressionParser.parseExpression("amount + id == 100"), + ExpressionParser.parseExpression("amount - id == 100"), + ExpressionParser.parseExpression("amount * id == 100"), + ExpressionParser.parseExpression("amount / id == 100"), + ExpressionParser.parseExpression("-amount == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractPostfixConditions(): Unit =

          { + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NULL, "('flag).isNull") + // IS_NOT_NULL will be eliminated since flag is not nullable + // testExtractSinglePostfixCondition(SqlStdOperatorTable.IS_NOT_NULL, "('flag).isNotNull") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_TRUE, "('flag).isTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_TRUE, "('flag).isNotTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_FALSE, "('flag).isFalse") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_FALSE, "('flag).isNotFalse") + }

          +
          + @Test
          + def testExtractConditionWithFunctionCalls(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // sum(amount) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCall(SqlStdOperatorTable.SUM, t0), t2)) + + // min(id) == 100 + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MIN, t1), t2)) + + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("sum(amount) > 100"), + ExpressionParser.parseExpression("min(id) == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + }

          +
          + @Test
          + def testExtractWithUnsupportedConditions(): Unit =

          { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // unsupported now: amount.cast(BigInteger) + val cast = builder.addExpr(rexBuilder.makeCast(allFieldTypes.get(1), t0)) + + // unsupported now: amount.cast(BigInteger) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCast(allFieldTypes.get(1), t0), t2)) + + // amount <= id + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // contains unsupported condition: (amount.cast(BigInteger) > 100 OR amount <= id) + val condition3 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.OR, condition1, condition2)) + + // only condition2 can be translated + builder.addCondition( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2, condition3)) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount <= id") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(2, unconvertedRexNodes.length) + assertEquals(">(CAST($2):BIGINT NOT NULL, 100)", unconvertedRexNodes(0).toString) + assertEquals("OR(>(CAST($2):BIGINT NOT NULL, 100), <=($2, $1))", + unconvertedRexNodes(1).toString) + }

          +
          + private def testExtractSinglePostfixCondition(
          + fieldIndex: Integer,
          — End diff –

          changed

          Show
          githubbot ASF GitHub Bot added a comment - Github user KurtYoung commented on a diff in the pull request: https://github.com/apache/flink/pull/3520#discussion_r106566599 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala — @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.util + +import java.math.BigDecimal + +import org.apache.calcite.rex. {RexBuilder, RexProgramBuilder} +import org.apache.calcite.sql.SqlPostfixOperator +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.flink.table.expressions. {Expression, ExpressionParser} +import org.apache.flink.table.validate.FunctionCatalog +import org.junit.Assert. {assertArrayEquals, assertEquals} +import org.junit.Test + +import scala.collection.JavaConverters._ + +class RexProgramExtractorTest extends RexProgramTestBase { + + private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns + + @Test + def testExtractRefInputFields(): Unit = { + val usedFields = RexProgramExtractor.extractRefInputFields(buildSimpleRexProgram1()) + assertArrayEquals(usedFields, Array(2, 3, 1)) + } + + @Test + def testExtractSimpleCondition(): Unit = { + val builder: RexBuilder = new RexBuilder(typeFactory) + val program = buildSimpleRexProgram2() + + val firstExp = ExpressionParser.parseExpression("id > 6") + val secondExp = ExpressionParser.parseExpression("amount * price < 100") + val expected: Array[Expression] = Array(firstExp, secondExp) + + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + builder, + functionCatalog) + + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractSingleCondition(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + + // a = amount >= id + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)) + builder.addCondition(a) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array(ExpressionParser.parseExpression("amount >= id")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + // ((a AND b) OR c) AND (NOT d) => (a OR c) AND (b OR c) AND (NOT d) + @Test + def testExtractCnfCondition(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // price + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) + // 100 + val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // a = amount < 100 + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t3)) + // b = id > 100 + val b = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t3)) + // c = price == 100 + val c = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t3)) + // d = amount <= id + val d = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // a AND b + val and = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(a, b).asJava)) + // (a AND b) or c + val or = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.OR, List(and, c).asJava)) + // not d + val not = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT, List(d).asJava)) + + // (a AND b) OR c) AND (NOT d) + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, List(or, not).asJava))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < 100 || price == 100"), + ExpressionParser.parseExpression("id > 100 || price == 100"), + ExpressionParser.parseExpression("!(amount <= id)")) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractArithmeticConditions(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + val condition = List( + // amount < id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t1)), + // amount <= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)), + // amount <> id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, t0, t1)), + // amount == id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t0, t1)), + // amount >= id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, t0, t1)), + // amount > id + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t0, t1)), + // amount + id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.PLUS, t0, t1), t2)), + // amount - id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MINUS, t0, t1), t2)), + // amount * id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t1), t2)), + // amount / id == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE, t0, t1), t2)), + // -amount == 100 + builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.UNARY_MINUS, t0), t2)) + ).asJava + + builder.addCondition(builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, condition))) + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < id"), + ExpressionParser.parseExpression("amount <= id"), + ExpressionParser.parseExpression("amount <> id"), + ExpressionParser.parseExpression("amount == id"), + ExpressionParser.parseExpression("amount >= id"), + ExpressionParser.parseExpression("amount > id"), + ExpressionParser.parseExpression("amount + id == 100"), + ExpressionParser.parseExpression("amount - id == 100"), + ExpressionParser.parseExpression("amount * id == 100"), + ExpressionParser.parseExpression("amount / id == 100"), + ExpressionParser.parseExpression("-amount == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractPostfixConditions(): Unit = { + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NULL, "('flag).isNull") + // IS_NOT_NULL will be eliminated since flag is not nullable + // testExtractSinglePostfixCondition(SqlStdOperatorTable.IS_NOT_NULL, "('flag).isNotNull") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_TRUE, "('flag).isTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_TRUE, "('flag).isNotTrue") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_FALSE, "('flag).isFalse") + testExtractSinglePostfixCondition(4, SqlStdOperatorTable.IS_NOT_FALSE, "('flag).isNotFalse") + } + + @Test + def testExtractConditionWithFunctionCalls(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // sum(amount) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCall(SqlStdOperatorTable.SUM, t0), t2)) + + // min(id) == 100 + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + rexBuilder.makeCall(SqlStdOperatorTable.MIN, t1), t2)) + + builder.addCondition(builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2))) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("sum(amount) > 100"), + ExpressionParser.parseExpression("min(id) == 100") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(0, unconvertedRexNodes.length) + } + + @Test + def testExtractWithUnsupportedConditions(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // 100 + val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // unsupported now: amount.cast(BigInteger) + val cast = builder.addExpr(rexBuilder.makeCast(allFieldTypes.get(1), t0)) + + // unsupported now: amount.cast(BigInteger) > 100 + val condition1 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, + rexBuilder.makeCast(allFieldTypes.get(1), t0), t2)) + + // amount <= id + val condition2 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // contains unsupported condition: (amount.cast(BigInteger) > 100 OR amount <= id) + val condition3 = builder.addExpr( + rexBuilder.makeCall(SqlStdOperatorTable.OR, condition1, condition2)) + + // only condition2 can be translated + builder.addCondition( + rexBuilder.makeCall(SqlStdOperatorTable.AND, condition1, condition2, condition3)) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + val (convertedExpressions, unconvertedRexNodes) = + RexProgramExtractor.extractConjunctiveConditions( + program, + relBuilder, + functionCatalog) + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount <= id") + ) + assertExpressionArrayEquals(expected, convertedExpressions) + assertEquals(2, unconvertedRexNodes.length) + assertEquals(">(CAST($2):BIGINT NOT NULL, 100)", unconvertedRexNodes(0).toString) + assertEquals("OR(>(CAST($2):BIGINT NOT NULL, 100), <=($2, $1))", + unconvertedRexNodes(1).toString) + } + + private def testExtractSinglePostfixCondition( + fieldIndex: Integer, — End diff – changed
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3520#discussion_r106566606

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala —
          @@ -0,0 +1,347 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specif