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

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

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

    https://github.com/apache/flink/pull/1939#discussion_r61120172
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchTableSourceScan.scala
 ---
    @@ -0,0 +1,139 @@
    +/*
    + * 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 org.apache.calcite.plan._
    +import org.apache.calcite.rel.RelNode
    +import org.apache.calcite.rel.`type`.RelDataType
    +import org.apache.calcite.rel.core.TableScan
    +import org.apache.calcite.rel.metadata.RelMetadataQuery
    +import org.apache.flink.api.common.functions.MapFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.DataSet
    +import org.apache.flink.api.java.typeutils.PojoTypeInfo
    +import org.apache.flink.api.table.BatchTableEnvironment
    +import org.apache.flink.api.table.codegen.CodeGenerator
    +import org.apache.flink.api.table.plan.schema.TableSourceTable
    +import org.apache.flink.api.table.runtime.MapRunner
    +import org.apache.flink.api.table.sources.BatchTableSource
    +import 
org.apache.flink.api.table.typeutils.TypeConverter.determineReturnType
    +
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +
    +/** Flink RelNode to read data from an external source defined by a 
[[BatchTableSource]]. */
    +class BatchTableSourceScan(
    --- End diff --
    
    I agree, both classes share a lot of code, but another difference apart 
from the unwrapping is that `DataSetScan` simply forwards its `DataSet` whereas 
`BatchTableSourceScan` creates a new `DataSet` using the `BatchTableSource` and 
the table environment. So I would like to keep separate classes, but it makes 
sense to let them have a common abstract base class. What do you think?
    Btw. `DataSetSource` was recently renamed to `DataSetScan` ;-)


> Add interfaces for Table API input formats
> ------------------------------------------
>
>                 Key: FLINK-2828
>                 URL: https://issues.apache.org/jira/browse/FLINK-2828
>             Project: Flink
>          Issue Type: New Feature
>          Components: Table API
>            Reporter: Timo Walther
>            Assignee: Fabian Hueske
>
> In order to support input formats for the Table API, interfaces are 
> necessary. I propose two types of TableSources:
> - AdaptiveTableSources can adapt their output to the requirements of the 
> plan. Although the output schema stays the same, the TableSource can react on 
> field resolution and/or predicates internally and can return adapted 
> DataSet/DataStream versions in the "translate" step.
> - StaticTableSources are an easy way to provide the Table API with additional 
> input formats without much implementation effort (e.g. for fromCsvFile())
> TableSources need to be deeply integrated into the Table API.
> The TableEnvironment requires a newly introduced AbstractExecutionEnvironment 
> (common super class of all ExecutionEnvironments for DataSets and 
> DataStreams).
> Here's what a TableSource can see from more complicated queries:
> {code}
> getTableJava(tableSource1)
>   .filter("a===5 || a===6")
>   .select("a as a4, b as b4, c as c4")
>   .filter("b4===7")
>   .join(getTableJava(tableSource2))
>   .where("a===a4 && c==='Test' && c4==='Test2'")
> // Result predicates for tableSource1:
> //  List("a===5 || a===6", "b===7", "c==='Test2'")
> // Result predicates for tableSource2:
> //  List("c==='Test'")
> // Result resolved fields for tableSource1 (true = filtering, 
> false=selection):
> //  Set(("a", true), ("a", false), ("b", true), ("b", false), ("c", false), 
> ("c", true))
> // Result resolved fields for tableSource2 (true = filtering, 
> false=selection):
> //  Set(("a", true), ("c", true))
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to