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

Jark Wu commented on FLINK-12848:
---------------------------------

Hi [~aloyszhang] [~yanghua] [~enricoc], thanks for reporting this. 

However, I don't think it's a safe way to compare field names in 
RowTypeInfo#equals. As [~enricoc] mentioned, we explicitly explained 
RowTypeInfo#equals should not check field names in FLINK-9444. Because 
RowTypeInfo is an information describes the structure of 
{{org.apache.flink.types.Row}}, there is no field names in Row. And in runtime 
(e.g. serialize/deserialize), the field names also don't matter.

We heavily used TypeInformation (including #equals) in SQL before. This led to 
a lot of problems. That's why [~tiwalter] introduced a well-defined type system 
in SQL recently. This work is still ongoing, and we will replace the 
TypeInformation with the new type system totally in SQL/Table. At that time, 
the problems you encountered should be fixed.

[~aloyszhang] could you attach the implementation of 
{{SimpleProcessionTimeSource}}, without it I can't reproduce the exception you 
mentioned. We can try it again to see whether it has been solved.


> Method equals() in RowTypeInfo should consider fieldsNames
> ----------------------------------------------------------
>
>                 Key: FLINK-12848
>                 URL: https://issues.apache.org/jira/browse/FLINK-12848
>             Project: Flink
>          Issue Type: Improvement
>          Components: Table SQL / Planner
>    Affects Versions: 1.7.2
>            Reporter: aloyszhang
>            Assignee: aloyszhang
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 0.5h
>  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