Skip to content

Commit bf1cd86

Browse files
authored
[hotfix] ELEMENT should throw TableRuntimeException
1 parent 6c3d8fc commit bf1cd86

File tree

3 files changed

+23
-13
lines changed

3 files changed

+23
-13
lines changed

flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1423,7 +1423,7 @@ object ScalarOperatorGens {
14231423
| $resultTerm = $nullTerm ? $defaultValue : $arrayGet;
14241424
| break;
14251425
| default:
1426-
| throw new RuntimeException("Array has more than one element.");
1426+
| throw new org.apache.flink.table.api.TableRuntimeException("Array has more than one element.");
14271427
|}
14281428
|""".stripMargin
14291429

flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java

Lines changed: 22 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020

2121
import org.apache.flink.table.annotation.DataTypeHint;
2222
import org.apache.flink.table.api.DataTypes;
23+
import org.apache.flink.table.api.TableRuntimeException;
2324
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
2425
import org.apache.flink.table.functions.ScalarFunction;
2526
import org.apache.flink.types.Row;
@@ -57,7 +58,8 @@ Stream<TestSetSpec> getTestSetSpecs() {
5758
arraySortTestCases(),
5859
arrayExceptTestCases(),
5960
arrayIntersectTestCases(),
60-
splitTestCases())
61+
splitTestCases(),
62+
arrayElementTestCases())
6163
.flatMap(s -> s);
6264
}
6365

@@ -1879,4 +1881,23 @@ private Stream<TestSetSpec> splitTestCases() {
18791881
"SPLIT(f1, '1', '2')",
18801882
"No match found for function signature SPLIT(<CHARACTER>, <CHARACTER>, <CHARACTER>)"));
18811883
}
1884+
1885+
private Stream<TestSetSpec> arrayElementTestCases() {
1886+
return Stream.of(
1887+
TestSetSpec.forFunction(BuiltInFunctionDefinitions.ARRAY_ELEMENT)
1888+
.onFieldsWithData(
1889+
new Integer[] {1}, new Integer[] {1, 2}, new float[] {4.0F}, null)
1890+
.andDataTypes(
1891+
DataTypes.ARRAY(DataTypes.INT()),
1892+
DataTypes.ARRAY(DataTypes.INT()),
1893+
DataTypes.ARRAY(DataTypes.FLOAT()),
1894+
DataTypes.ARRAY(DataTypes.INT()))
1895+
.testResult($("f0").element(), "ELEMENT(f0)", 1, DataTypes.INT())
1896+
.testSqlRuntimeError(
1897+
"ELEMENT(f1)",
1898+
TableRuntimeException.class,
1899+
"Array has more than one element.")
1900+
.testResult($("f2").element(), "ELEMENT(f2)", 4.0F, DataTypes.FLOAT())
1901+
.testResult($("f3").element(), "ELEMENT(f3)", null, DataTypes.INT()));
1902+
}
18821903
}

flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala

Lines changed: 0 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -190,17 +190,6 @@ class ArrayTypeTest extends ArrayTypeTestBase {
190190

191191
testAllApis('f11.cardinality(), "CARDINALITY(f11)", "1")
192192

193-
// element
194-
testAllApis('f9.element(), "ELEMENT(f9)", "1")
195-
196-
testAllApis('f8.element(), "ELEMENT(f8)", "4.0")
197-
198-
testAllApis('f10.element(), "ELEMENT(f10)", "NULL")
199-
200-
testAllApis('f4.element(), "ELEMENT(f4)", "NULL")
201-
202-
testAllApis('f11.element(), "ELEMENT(f11)", "1")
203-
204193
// comparison
205194
testAllApis('f2 === 'f5.at(1), "f2 = f5[1]", "TRUE")
206195

0 commit comments

Comments
 (0)