[ https://issues.apache.org/jira/browse/FLINK-31835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17794179#comment-17794179 ]
Timo Walther commented on FLINK-31835: -------------------------------------- {quote} My only concern is the compatibility problem. What do you think Timo Walther? Is there any potential problems? {quote} Sorry that I did not answer earlier [~jark]. But my answer would have been yes, there are compatibility problems. Hints are not perfect. Sometimes a user needs to override the {{getTypeInference()}} with {code} outputTypeStrategy(() -> DataTypes.ARRAY(INT().notNull()).bringedTo(int[].class)) {code} if a class other that the default class should be used. The default class of INT is Integer, thus the default class for array is Integer[]. > DataTypeHint don't support Row<i Array<int>> > -------------------------------------------- > > Key: FLINK-31835 > URL: https://issues.apache.org/jira/browse/FLINK-31835 > Project: Flink > Issue Type: Bug > Components: Table SQL / API > Affects Versions: 1.15.4 > Reporter: jeff-zou > Assignee: Aitozi > Priority: Major > Labels: pull-request-available > Fix For: 1.18.0 > > > Using DataTypeHint("Row<t ARRAY<INT>>") in a UDF gives the following error: > > {code:java} > Caused by: java.lang.ClassCastException: class [I cannot be cast to class > [Ljava.lang.Object; ([I and [Ljava.lang.Object; are in module java.base of > loader 'bootstrap') > org.apache.flink.table.data.conversion.ArrayObjectArrayConverter.toInternal(ArrayObjectArrayConverter.java:40) > org.apache.flink.table.data.conversion.DataStructureConverter.toInternalOrNull(DataStructureConverter.java:61) > org.apache.flink.table.data.conversion.RowRowConverter.toInternal(RowRowConverter.java:75) > org.apache.flink.table.data.conversion.RowRowConverter.toInternal(RowRowConverter.java:37) > org.apache.flink.table.data.conversion.DataStructureConverter.toInternalOrNull(DataStructureConverter.java:61) > StreamExecCalc$251.processElement_split9(Unknown Source) > StreamExecCalc$251.processElement(Unknown Source) > org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:82) > {code} > > The function is as follows: > {code:java} > @DataTypeHint("Row<t ARRAY<INT>>") > public Row eval() { > int[] i = new int[3]; > return Row.of(i); > } {code} > > This error is not reported when testing other simple types, so it is not an > environmental problem. -- This message was sent by Atlassian Jira (v8.20.10#820010)