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

Andriy Onyshchuk commented on FLINK-33200:
------------------------------------------

Seems like I have caught several more issues related to `AT` expression and 
type resolution.
 # accessing array of primitives doesn't work if at data level arrays 
represented as ArrayData. What I got is: `Unsupported conversion from data type 
'ARRAY<INT>' (conversion class: org.apache.flink.table.data.ArrayData) to type 
information. Only data types that originated from type information fully 
support a reverse conversion.`
 # accessing `Map<Long, String>` throws too. `map.at(0L)` gets failed with 
Incompatible types for sink column
{{Exception in thread "main" org.apache.flink.table.api.ValidationException: 
Column types of query result and sink for '*anonymous_collect$3*' do not 
match.}}
{{Cause: Incompatible types for sink column 'longData_at_0' at position 0.}}
{{Query schema: [longData_at_0: STRING]}}
{{Sink schema: [longData_at_0: RAW('org.apache.flink.table.data.StringData', 
?)]}}

All issues are reflected in `IssueDemo.java` (see attachments).

> 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)

Reply via email to