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

Fabian Hueske commented on FLINK-5280:
--------------------------------------

Hi [~ivan.mushketyk],

First of all, you are completely right about the motivation for this issue. The 
goal is to add support for TableSources that return nested data just as in your 
example.
The Table API and SQL do support accessing nested fields since FLINK-4249 as 
[~jark] mentioned (see "Value access functions" at the end of the Built-In SQL 
function in the documentation as well).

[~jark] described that only the first level is flattened, i.e., the functions 
do not handle custom POJOs but only Rows (with possibly nested Pojos).

The actual challenge is that the TableSource would need to provide names for 
the nested fields. This is not possible with the current interface (without 
some implicit conventions) because the names are returned as flat String array.
Take the example of a TableSource that reads Avro data. Either the data is read 
as specific object, then we are good, because these are concrete, 
code-generated POJOs with the Table API can directly process. In case of a 
generic record, the data is hold in a {{Map<String,String>>}} and accessed 
using a schema. A TableSource would need to convert such a generic record into 
a nested data structure (for example nested {{Row}}s) and assign the names for 
the nested attributes. This is currently not possible. 

One possible solution would be to add support for names in {{RowTypeInfo}} and 
only declare the names of the first level attributes via the current interface 
and all nested field via the {{RowTypeInfo}}.
What I don't like about this approach is that the field names are hidden in the 
{{TypeInformation}}. But maybe that's the best approach because we can keep the 
interface and every TableSource could fall back to returning rows.


> 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