Prabhu Joseph created FLINK-33523: ------------------------------------- Summary: DataType ARRAY<INT NOT NULL> fails to cast into Object[] Key: FLINK-33523 URL: https://issues.apache.org/jira/browse/FLINK-33523 Project: Flink Issue Type: Bug Affects Versions: 1.18.0 Reporter: Prabhu Joseph
When upgrading Iceberg's Flink version to 1.18, we found the Flink-related unit test case broken due to this issue. The below code used to work fine in Flink 1.17 but failed after upgrading to 1.18. DataType ARRAY<INT NOT NULL> fails to cast into Object[]. **Error:** {code} Exception in thread "main" java.lang.ClassCastException: [I cannot be cast to [Ljava.lang.Object; at FlinkArrayIntNotNullTest.main(FlinkArrayIntNotNullTest.java:18) {code} **Repro:** {code} import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; public class FlinkArrayIntNotNullTest { public static void main(String[] args) throws Exception { EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); TableEnvironment env = TableEnvironment.create(settings); env.executeSql("CREATE TABLE filesystemtable2 (id INT, data ARRAY<INT NOT NULL>) WITH ('connector' = 'filesystem', 'path' = '/tmp/FLINK/filesystemtable2', 'format'='json')"); env.executeSql("INSERT INTO filesystemtable2 VALUES (4,ARRAY [1,2,3])"); TableResult tableResult = env.executeSql("SELECT * from filesystemtable2"); ArrayData actualArrayData = new GenericArrayData((Object[]) tableResult.collect().next().getField(1)); } } {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)