[
https://issues.apache.org/jira/browse/FLINK-3848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705337#comment-15705337
]
ASF GitHub Bot commented on FLINK-3848:
---------------------------------------
Github user tonycox commented on a diff in the pull request:
https://github.com/apache/flink/pull/2810#discussion_r90010696
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/BatchProjectableTableSourceScanRule.scala
---
@@ -0,0 +1,85 @@
+/*
+ * 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.api.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.RelOptRule._
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.logical.{LogicalProject, LogicalTableScan}
+import org.apache.calcite.rex.RexInputRef
+import
org.apache.flink.api.table.plan.nodes.dataset.{BatchProjectableTableSourceScan,
DataSetConvention}
+import org.apache.flink.api.table.plan.schema.TableSourceTable
+import org.apache.flink.api.table.sources.{BatchTableSource,
ProjectableTableSource}
+
+import scala.collection.JavaConverters._
+
+/** Rule to convert a [[LogicalTableScan]] with [[LogicalProject]]
+ * into a [[BatchProjectableTableSourceScan]].
+ */
+class BatchProjectableTableSourceScanRule
+ extends RelOptRule(
+ operand(classOf[LogicalProject], operand(classOf[TableScan], none())),
+ "BatchProjectableTableSourceScanRule") {
+
+ /** Rule must only match if TableScan targets a [[BatchTableSource]],
+ * LogicalProject targets a [[ProjectableTableSource]] and all operands
are [[RexInputRef]]
+ */
+ override def matches(call: RelOptRuleCall): Boolean = {
+ val project: LogicalProject = call.rel(0).asInstanceOf[LogicalProject]
+ val scan: TableScan = call.rel(1).asInstanceOf[TableScan]
+ val dataSetTable = scan.getTable.unwrap(classOf[TableSourceTable])
+ dataSetTable match {
+ case tst: TableSourceTable =>
+ tst.tableSource match {
+ case s: BatchTableSource[_] =>
+ s match {
+ case p: ProjectableTableSource[_] =>
+
project.getProjects.asScala.forall(_.isInstanceOf[RexInputRef])
+ case _ =>
+ false
+ }
+ case _ =>
+ false
+ }
+ case _ =>
+ false
+ }
+ }
+
+ override def onMatch(call: RelOptRuleCall): Unit = {
+ val project = call.rel(0).asInstanceOf[LogicalProject]
+ val scan: TableScan = call.rel(1).asInstanceOf[TableScan]
+
+ val convInput = RelOptRule.convert(scan, DataSetConvention.INSTANCE)
+ val traitSet: RelTraitSet =
project.getTraitSet.replace(DataSetConvention.INSTANCE)
+
+ val newRel = new BatchProjectableTableSourceScan(
--- End diff --
How can I adapted RexProgram in the same Rule?
use `transformTo()` for `BatchProjectableTableSourceScan` and then for
custom `LogicalCalc` ?
> Add ProjectableTableSource interface and translation rule
> ---------------------------------------------------------
>
> Key: FLINK-3848
> URL: https://issues.apache.org/jira/browse/FLINK-3848
> Project: Flink
> Issue Type: New Feature
> Components: Table API & SQL
> Reporter: Fabian Hueske
> Assignee: Anton Solovev
>
> Add a {{ProjectableTableSource}} interface for {{TableSource}} implementation
> that support projection push-down.
> The interface could look as follows
> {code}
> def trait ProjectableTableSource {
> def setProjection(fields: Array[String]): Unit
> }
> {code}
> In addition we need Calcite rules to push a projection into a TableScan that
> refers to a {{ProjectableTableSource}}. We might need to tweak the cost model
> as well to push the optimizer in the right direction.
> Moreover, the {{CsvTableSource}} could be extended to implement
> {{ProjectableTableSource}}.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)