[
https://issues.apache.org/jira/browse/FLINK-3849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15878675#comment-15878675
]
ASF GitHub Bot commented on FLINK-3849:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/3166#discussion_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
> Add FilterableTableSource interface and translation rule
> --------------------------------------------------------
>
> Key: FLINK-3849
> URL: https://issues.apache.org/jira/browse/FLINK-3849
> Project: Flink
> Issue Type: New Feature
> Components: Table API & SQL
> Reporter: Fabian Hueske
> Assignee: Anton Solovev
>
> Add a {{FilterableTableSource}} interface for {{TableSource}} implementations
> which support filter push-down.
> The interface could look as follows
> {code}
> def trait FilterableTableSource {
> // returns unsupported predicate expression
> def setPredicate(predicate: Expression): Expression
> }
> {code}
> In addition we need Calcite rules to push a predicate (or parts of it) into a
> TableScan that refers to a {{FilterableTableSource}}. We might need to tweak
> the cost model as well to push the optimizer in the right direction.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)