[ https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17773908#comment-17773908 ]
xuyang commented on FLINK-33200: -------------------------------- Hi, [~andyglow] . I think the issue 1 and 2 are caused by different bugs. The issue 1 is caused by that the field 'events' refs from the table, and its actual type is StringData, converted from the origin type DataTypes.STRING (). But the key of the 'AT' named 'eventType' is DataType.STRING, not StringData. So the difference about these two types causes the failure about validating 'AT'. The issue 2 is caused by that if the planner can't infer the result type of the table expression (that means the result type is Polymorphic, depending on the type of args), the legacy rule is used to infer the result type. But the legacy rules wrongly create a LegacyTypeInformation for StringData. Not only 'AT', but also 'COLLECT' (inTable.select(Expressions.$("id").collect()).execute().print();) will cause this bug. I'll try to fix all of them. > ItemAt Expression validation fail in Table API due to type mismatch > ------------------------------------------------------------------- > > Key: FLINK-33200 > URL: https://issues.apache.org/jira/browse/FLINK-33200 > Project: Flink > Issue Type: Bug > Components: Table SQL / API > Affects Versions: 1.18.0, 1.17.1 > Reporter: Zhenqiu Huang > Priority: Minor > Attachments: IssueDemo.java > > > The table schema is defined as below: > public static final DataType DATA_TYPE = DataTypes.ROW( > DataTypes.FIELD("id", DataTypes.STRING()), > DataTypes.FIELD("events", > DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING()))) > ); > public static final Schema SCHEMA = > Schema.newBuilder().fromRowDataType(DATA_TYPE).build(); > inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType") > The validation fail as "eventType" is inferred as > BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a > StringDataTypeInfo. The validation fail at > case mti: MapTypeInfo[_, _] => > if (key.resultType == mti.getKeyTypeInfo) { > ValidationSuccess > } else { > ValidationFailure( > s"Map entry access needs a valid key of type " + > s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.") > } -- This message was sent by Atlassian Jira (v8.20.10#820010)