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

Enrico Canzonieri commented on FLINK-12848:
-------------------------------------------

This is causing issues to one of our queries where the schema has two nested 
records that have two fields of the same type but different name, e.g. 
Row(Row(a: Int, b: Int), Row(c: Int, d:Int)) where "a", "b", "c", "d" are the 
field names.

The code in 
[https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala#L92|https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala#L92)]
 that is caching the Type conversion is returning the for Row(c: Int, d: Int) 
the conversion cached for the first nested Row. As result the generated table 
schema will have mixed up (and clashing) field names.

I see that the equals() change to RowTypeInfo was introduced in FLINK-9444. Is 
there any reason why we should never consider the field names for RowTypeInfo 
equals? If so would it make sense to have a method that covers that special (to 
my understanding) case and make equals also include names?

I'm currently planning to fix this locally by extending the equals method of 
RowTypeInfo, but it'd be great to know whether that's safe to do or not.

> Method equals() in RowTypeInfo should consider fieldsNames
> ----------------------------------------------------------
>
>                 Key: FLINK-12848
>                 URL: https://issues.apache.org/jira/browse/FLINK-12848
>             Project: Flink
>          Issue Type: Improvement
>    Affects Versions: 1.7.2
>            Reporter: aloyszhang
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.9.0
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> Since the `RowTypeInfo#equals()` does not consider the fieldNames , when 
> process data with RowTypeInfo type there may comes an error of the field 
> name.  
> {code:java}
> String [] fields = new String []{"first", "second"};
> TypeInformation<?>[] types = new TypeInformation[]{
> Types.ROW_NAMED(new String[]{"first001"}, Types.INT),
> Types.ROW_NAMED(new String[]{"second002"}, Types.INT) }; 
> StreamExecutionEnvironment execEnv = 
> StreamExecutionEnvironment.getExecutionEnvironment();
> StreamTableEnvironment env = 
> StreamTableEnvironment.getTableEnvironment(execEnv);
> SimpleProcessionTimeSource streamTableSource = new 
> SimpleProcessionTimeSource(fields, types);
> env.registerTableSource("testSource", streamTableSource);
> Table sourceTable = env.scan("testSource");
> System.out.println("Source table schema : ");
> sourceTable.printSchema();
> {code}
> The table shcema will be 
> {code:java}
> Source table schema : 
> root 
> |-- first: Row(first001: Integer) 
> |-- second: Row(first001: Integer) 
> |-- timestamp: TimeIndicatorTypeInfo(proctime)
> {code}
> the second field has the same name with the first field.
> So, we should consider the fieldnames in RowTypeInfo#equals()
>  
>  
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to