[
https://issues.apache.org/jira/browse/FLINK-5280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15775054#comment-15775054
]
ASF GitHub Bot commented on FLINK-5280:
---------------------------------------
Github user wuchong commented on a diff in the pull request:
https://github.com/apache/flink/pull/3039#discussion_r93816451
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
---
@@ -566,17 +574,13 @@ object TableEnvironment {
def getFieldTypes(inputType: TypeInformation[_]):
Array[TypeInformation[_]] = {
validateType(inputType)
- inputType match {
- case t: TupleTypeInfo[_] => getTypes(t)
- case c: CaseClassTypeInfo[_] => getTypes(c)
- case p: PojoTypeInfo[_] => getTypes(p)
- case r: RowTypeInfo => getTypes(r)
- case tpe =>
- throw new TableException(s"Type $tpe lacks explicit field naming")
+ getFieldNames(inputType).map { i =>
--- End diff --
I think this maybe error-prone. Field types array should be mapped by
indices not field names. Such as PojoType filed names' order is not equal to
field types' order. The original code in `UserDefinedFunctionUtil.getFieldInfo`
maybe wrong too.
> 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)