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

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

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

    https://github.com/apache/flink/pull/2810#discussion_r89754439
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchProjectableTableSourceScan.scala
 ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.nodes.dataset
    +
    +import java.util
    +
    +import org.apache.calcite.plan._
    +import org.apache.calcite.rel.RelNode
    +import org.apache.calcite.rel.`type`.RelDataType
    +import org.apache.calcite.rel.metadata.RelMetadataQuery
    +import org.apache.calcite.rex.{RexInputRef, RexNode}
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.DataSet
    +import org.apache.flink.api.table.BatchTableEnvironment
    +import org.apache.flink.api.table.plan.schema.TableSourceTable
    +import org.apache.flink.api.table.sources.{BatchTableSource, 
ProjectableTableSource}
    +
    +import scala.collection.JavaConverters._
    +
    +class BatchProjectableTableSourceScan(
    +    cluster: RelOptCluster,
    +    traits: RelTraitSet,
    +    input: RelNode,
    +    projects: util.List[_ <: RexNode],
    +    projectionRowType: RelDataType,
    +    table: RelOptTable)
    +  extends BatchScan(cluster, traits, table, projectionRowType) {
    +
    +  val tableSourceTable = getTable.unwrap(classOf[TableSourceTable])
    +  val projectableSource = 
tableSourceTable.tableSource.asInstanceOf[ProjectableTableSource[_]]
    +
    +  override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): 
RelNode = {
    +    new BatchProjectableTableSourceScan(
    +      cluster,
    +      traitSet,
    +      input,
    +      projects,
    +      projectionRowType,
    +      getTable)
    +  }
    +
    +  override def computeSelfCost(planner: RelOptPlanner, mq: 
RelMetadataQuery): RelOptCost = {
    --- End diff --
    
    I think we should adapt the costs a bit. Right now, there are no IO costs 
assumed and only the cardinality but not the row width are reflected.
    
    I propose the following:
    - move the implementation of `BatchScan.computeSelfCost()` to 
`DataSetScan`. Here IO costs are `0`.
    - implement `BatchTableSourceScan.computeSelfCost()` to have IO costs of 
`rowCnt * super.estimateRowSize()`.



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

Reply via email to