flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-3849) Add FilterableTableSource interface and translation rule
Date Fri, 24 Feb 2017 06:27:44 GMT

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

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

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.


> 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)

Mime
View raw message