This is an automated email from the ASF dual-hosted git repository. snuyanzin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push: new bf1cd860617 [hotfix] `ELEMENT` should throw `TableRuntimeException` bf1cd860617 is described below commit bf1cd860617f7b51ac91516814c0e931e5bba241 Author: Natea Eshetu Beshada <nbeshada+...@confluent.io> AuthorDate: Mon Jul 21 22:24:53 2025 -0700 [hotfix] `ELEMENT` should throw `TableRuntimeException` --- .../planner/codegen/calls/ScalarOperatorGens.scala | 2 +- .../functions/CollectionFunctionsITCase.java | 23 +++++++++++++++++++++- .../table/planner/expressions/ArrayTypeTest.scala | 11 ----------- 3 files changed, 23 insertions(+), 13 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 686c71c7dda..a82eeb7280e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -1423,7 +1423,7 @@ object ScalarOperatorGens { | $resultTerm = $nullTerm ? $defaultValue : $arrayGet; | break; | default: - | throw new RuntimeException("Array has more than one element."); + | throw new org.apache.flink.table.api.TableRuntimeException("Array has more than one element."); |} |""".stripMargin diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java index 230bf290c0a..f67d422c994 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.functions; import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableRuntimeException; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.types.Row; @@ -57,7 +58,8 @@ class CollectionFunctionsITCase extends BuiltInFunctionTestBase { arraySortTestCases(), arrayExceptTestCases(), arrayIntersectTestCases(), - splitTestCases()) + splitTestCases(), + arrayElementTestCases()) .flatMap(s -> s); } @@ -1879,4 +1881,23 @@ class CollectionFunctionsITCase extends BuiltInFunctionTestBase { "SPLIT(f1, '1', '2')", "No match found for function signature SPLIT(<CHARACTER>, <CHARACTER>, <CHARACTER>)")); } + + private Stream<TestSetSpec> arrayElementTestCases() { + return Stream.of( + TestSetSpec.forFunction(BuiltInFunctionDefinitions.ARRAY_ELEMENT) + .onFieldsWithData( + new Integer[] {1}, new Integer[] {1, 2}, new float[] {4.0F}, null) + .andDataTypes( + DataTypes.ARRAY(DataTypes.INT()), + DataTypes.ARRAY(DataTypes.INT()), + DataTypes.ARRAY(DataTypes.FLOAT()), + DataTypes.ARRAY(DataTypes.INT())) + .testResult($("f0").element(), "ELEMENT(f0)", 1, DataTypes.INT()) + .testSqlRuntimeError( + "ELEMENT(f1)", + TableRuntimeException.class, + "Array has more than one element.") + .testResult($("f2").element(), "ELEMENT(f2)", 4.0F, DataTypes.FLOAT()) + .testResult($("f3").element(), "ELEMENT(f3)", null, DataTypes.INT())); + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala index bc4b13615e3..a0bc8709c06 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala @@ -190,17 +190,6 @@ class ArrayTypeTest extends ArrayTypeTestBase { testAllApis('f11.cardinality(), "CARDINALITY(f11)", "1") - // element - testAllApis('f9.element(), "ELEMENT(f9)", "1") - - testAllApis('f8.element(), "ELEMENT(f8)", "4.0") - - testAllApis('f10.element(), "ELEMENT(f10)", "NULL") - - testAllApis('f4.element(), "ELEMENT(f4)", "NULL") - - testAllApis('f11.element(), "ELEMENT(f11)", "1") - // comparison testAllApis('f2 === 'f5.at(1), "f2 = f5[1]", "TRUE")