This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.13
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.13 by this push:
     new a6a4ea5  [FLINK-22109][table-planner-blink] Resolve misleading 
exception message in invalid nested function
a6a4ea5 is described below

commit a6a4ea501bfb25667b67f3eacee47140766fbc4f
Author: MaChengLong <592577...@qq.com>
AuthorDate: Thu Apr 8 13:52:36 2021 +0800

    [FLINK-22109][table-planner-blink] Resolve misleading exception message in 
invalid nested function
    
    This closes #15523.
---
 .../planner/functions/inference/TypeInferenceOperandInference.java     | 3 ++-
 .../org/apache/flink/table/planner/functions/MiscFunctionsITCase.java  | 3 +++
 2 files changed, 5 insertions(+), 1 deletion(-)

diff --git 
a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java
 
b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java
index 16793e6..5e6e3d8 100644
--- 
a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java
+++ 
b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java
@@ -30,6 +30,7 @@ import 
org.apache.flink.table.types.inference.TypeInferenceUtil;
 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;
 
@@ -68,7 +69,7 @@ public final class TypeInferenceOperandInference implements 
SqlOperandTypeInfere
                 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);
diff --git 
a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
 
b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
index 9915b80..3fbce28 100644
--- 
a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
+++ 
b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
@@ -74,6 +74,9 @@ public class MiscFunctionsITCase extends 
BuiltInFunctionTestBase {
                                 "IFNULL(f1, f0)",
                                 new BigDecimal("123.45"),
                                 DataTypes.DECIMAL(12, 2).notNull())
+                        .testSqlError(
+                                "IFNULL(SUBSTR(''), f0)",
+                                "Invalid number of arguments to function 
'SUBSTR'.")
                         .testResult(
                                 $("f1").ifNull($("f0")),
                                 "IFNULL(f1, f0)",

Reply via email to