Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-22109][table-planner-blink] Resolve misleading exception message in invalid nested function #15523

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.flink.table.types.logical.LogicalType;

import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.runtime.CalciteContextException;
import org.apache.calcite.sql.SqlCallBinding;
import org.apache.calcite.sql.type.SqlOperandTypeInference;

Expand Down Expand Up @@ -68,7 +69,7 @@ public void inferOperandTypes(
new CallBindingCallContext(dataTypeFactory, definition, callBinding, returnType);
try {
inferOperandTypesOrError(unwrapTypeFactory(callBinding), callContext, operandTypes);
} catch (ValidationException e) {
} catch (ValidationException | CalciteContextException e) {
// let operand checker fail
} catch (Throwable t) {
throw createUnexpectedException(callContext, t);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ package org.apache.flink.table.planner.expressions
import org.apache.flink.table.api._
import org.apache.flink.table.expressions.{Expression, ExpressionParser, TimeIntervalUnit, TimePointUnit}
import org.apache.flink.table.planner.expressions.utils.ScalarTypesTestBase

import org.hamcrest.core.StringContains
import org.junit.Test

class ScalarFunctionsTest extends ScalarTypesTestBase {
Expand Down Expand Up @@ -4133,4 +4133,14 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
testSqlApi(s"IFNULL(CAST(INTERVAL '2' DAY AS VARCHAR(20)), $str2)", "+2 00:00:00.000")
testSqlApi(s"IFNULL(CAST(f53 AS VARCHAR(100)), $str2)", "hello world")
}

@Test
def testInvalidNestedFunction(): Unit = {
thrown.expect(classOf[ValidationException])
thrown.expectMessage(StringContains.containsString(
"""Invalid number of arguments to function 'SUBSTR'"""))
testSqlApi(
"IFNULL(SUBSTR('abc'), 'def')",
"abc")
}
}