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

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

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

    https://github.com/apache/flink/pull/3039#discussion_r94851171
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSource.scala
 ---
    @@ -19,21 +19,28 @@
     package org.apache.flink.table.sources
     
     import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.table.api.TableEnvironment
     
    -/** Defines an external table by providing schema information, i.e., field 
names and types.
    +/** Defines an external table by providing schema information and used to 
produce a
    +  * [[org.apache.flink.api.scala.DataSet]] or 
[[org.apache.flink.streaming.api.scala.DataStream]].
    +  * Schema information consists of a data type, field names, and 
corresponding indices of
    +  * these names in the data type.
    +  *
    +  * To define a TableSource one need to implement 
[[TableSource#getReturnType]]. In this case
    +  * field names and field indices are derived from the returned type.
    +  *
    +  * In case if custom field names are required one need to implement both
    +  * [[TableSource#getFieldsNames]] and [[TableSource#getFieldsIndices]].
       *
       * @tparam T The return type of the [[TableSource]].
       */
     trait TableSource[T] {
    --- End diff --
    
    Please rename `getFieldsNames()` to  `getFieldNames()` (the original 
`getFieldsNames() looks like a typo to me.`) and `getFieldsIndicies()` to 
`getFieldIndicies()`.


> Extend TableSource to support nested data
> -----------------------------------------
>
>                 Key: FLINK-5280
>                 URL: https://issues.apache.org/jira/browse/FLINK-5280
>             Project: Flink
>          Issue Type: Improvement
>          Components: Table API & SQL
>    Affects Versions: 1.2.0
>            Reporter: Fabian Hueske
>            Assignee: Ivan Mushketyk
>
> The {{TableSource}} interface does currently only support the definition of 
> flat rows. 
> However, there are several storage formats for nested data that should be 
> supported such as Avro, Json, Parquet, and Orc. The Table API and SQL can 
> also natively handle nested rows.
> The {{TableSource}} interface and the code to register table sources in 
> Calcite's schema need to be extended to support nested data.



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

Reply via email to