[ https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17773962#comment-17773962 ]
xuyang commented on FLINK-33200: -------------------------------- Hi, [~ZhenqiuHuang] , can you provide the code caused this issue? >From the example provided by [~andyglow] , I found that an internal >TypeInformation 'InternalTypeInfo' which should not be used actually is used. >After replacing 'InternalTypeInfo' with 'ExternalTypeInfo', issue 1 and 2 all >work fine. The new example can be found following. The main changes is: # use 'ExternalTypeInfo.of(DATA_TYPE)' instead of 'InternalTypeInfo.of(DATA_TYPE.getLogicalType())' # remove 'ArrayData', 'GenericMapData' and 'StringData' # use 'Row' instead of 'RowData' {code:java} public class DemoTest { public static class DataEnvelope { public final String id; public final List<Map<String, String>> events; public DataEnvelope(String id, List<Map<String, String>> events) { this.id = id; this.events = events; } public Row toRowData() { final Row row = new Row(2); row.setField(0, id); // row.setField(1, eventsMapData()); row.setField(1, events.toArray()); return row; } 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 TypeInformation<RowData> TYPE_INFO = ExternalTypeInfo.of(DATA_TYPE); public static final Schema SCHEMA = Schema.newBuilder().fromRowDataType(DATA_TYPE).build(); } public static final List<DataEnvelope> TEST_DATA = Lists.newArrayList( new DataEnvelope( "1", Lists.newArrayList(Maps.newHashMap("eventType", "enter"))), new DataEnvelope( "2", Lists.newArrayList( Maps.newHashMap("eventType", "dialog"), Maps.newHashMap("eventType", "exit")))); public static void main(String[] args) throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); final StreamTableEnvironment tenv = StreamTableEnvironment.create(env); final List inRows = TEST_DATA.stream().map(DataEnvelope::toRowData).collect(Collectors.toList()); final DataStreamSource inStream = env.fromCollection(inRows, DataEnvelope.TYPE_INFO); final Table inTable = tenv.fromDataStream(inStream, DataEnvelope.SCHEMA); // issue #1 inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")) .execute() .print(); // issue #2 inTable.select(Expressions.$("events").at(1).as("firstEvent")).execute().print(); env.close(); } } {code} > 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)