[ https://issues.apache.org/jira/browse/FLINK-6886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16047963#comment-16047963 ]
ASF GitHub Bot commented on FLINK-6886: --------------------------------------- Github user wuchong commented on the issue: https://github.com/apache/flink/pull/4102 I think the root cause is that TUMBLE_START inherits the wrong type (`TimeIndicatorRelDataType`) from rowtime column. So maybe a better solution is to re-create a `RelDataType` from `optimizedPlan`'s rowType and `orignalPlan`'s rowType and pass into the `getConversionMapper` method. I create a simple commit for this: https://github.com/wuchong/flink/commit/82c17ab45699f5f9beb925b156e760ebdeff79fb What do you think? @sunjincheng121 @fhueske BTW, do we really need so many IT cases for this ? > Fix Timestamp field can not be selected in event time case when > toDataStream[T], `T` not a `Row` Type. > ------------------------------------------------------------------------------------------------------- > > Key: FLINK-6886 > URL: https://issues.apache.org/jira/browse/FLINK-6886 > Project: Flink > Issue Type: Bug > Components: Table API & SQL > Affects Versions: 1.4.0 > Reporter: sunjincheng > Assignee: sunjincheng > > Currently for event-time window(group/over), When contain `Timestamp` type > field in `SELECT Clause`, And toDataStream[T], `T` not a `Row` Type, Such > `PojoType`, will throw a exception. In this JIRA. will fix this bug. For > example: > Group Window on SQL: > {code} > SELECT name, max(num) as myMax, TUMBLE_START(rowtime, INTERVAL '5' SECOND) as > winStart,TUMBLE_END(rowtime, INTERVAL '5' SECOND) as winEnd FROM T1 GROUP BY > name, TUMBLE(rowtime, INTERVAL '5' SECOND) > {code} > Throw Exception: > {code} > org.apache.flink.table.api.TableException: The field types of physical and > logical row types do not match.This is a bug and should not happen. Please > file an issue. > at org.apache.flink.table.api.TableException$.apply(exceptions.scala:53) > at > org.apache.flink.table.api.TableEnvironment.generateRowConverterFunction(TableEnvironment.scala:721) > at > org.apache.flink.table.api.StreamTableEnvironment.getConversionMapper(StreamTableEnvironment.scala:247) > at > org.apache.flink.table.api.StreamTableEnvironment.translate(StreamTableEnvironment.scala:647) > {code} > In fact, when we solve this exception,subsequent other exceptions will be > thrown. The real reason is {{TableEnvironment#generateRowConverterFunction}} > method bug. So in this JIRA. will fix it. -- This message was sent by Atlassian JIRA (v6.4.14#64029)