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

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


The following commit(s) were added to refs/heads/release-1.11 by this push:
     new 686410d  [FLINK-18628][table-common] Fix error message for overloaded 
function with same parameter names
686410d is described below

commit 686410dec29b00c04021dc4ebb39f116af68964a
Author: Timo Walther <twal...@apache.org>
AuthorDate: Fri Jul 17 15:00:11 2020 +0200

    [FLINK-18628][table-common] Fix error message for overloaded function with 
same parameter names
    
    This closes #12928.
---
 .../flink/table/types/inference/TypeInferenceUtil.java       |  2 +-
 .../flink/table/types/inference/InputTypeStrategiesTest.java | 12 +++++++++++-
 2 files changed, 12 insertions(+), 2 deletions(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java
index e4c09da..571ef5d 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java
@@ -170,7 +170,7 @@ public final class TypeInferenceUtil {
                        TypeInference typeInference,
                        String name,
                        FunctionDefinition definition) {
-               if (typeInference.getNamedArguments().isPresent() || 
typeInference.getTypedArguments().isPresent()) {
+               if (typeInference.getTypedArguments().isPresent()) {
                        return formatNamedOrTypedArguments(name, typeInference);
                }
                return 
typeInference.getInputTypeStrategy().getExpectedSignatures(definition)
diff --git 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
index 7bdc29c..3a6fa6b 100644
--- 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
+++ 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
@@ -562,7 +562,17 @@ public class InputTypeStrategiesTest {
                                .calledWithArgumentTypes(DataTypes.FLOAT())
                                .expectSignature("f(<EXACT_NUMERIC>)")
                                .expectErrorMessage(
-                                       "Unsupported argument type. Expected 
type of family 'EXACT_NUMERIC' but actual type was 'FLOAT'.")
+                                       "Unsupported argument type. Expected 
type of family 'EXACT_NUMERIC' but actual type was 'FLOAT'."),
+
+                       TestSpec
+                               .forStrategy(
+                                       "Same named arguments for overloaded 
method.",
+                                       or(
+                                               
sequence(explicit(DataTypes.STRING())),
+                                               
sequence(explicit(DataTypes.INT()))))
+                               .namedArguments("sameName")
+                               .calledWithArgumentTypes(DataTypes.BOOLEAN())
+                               .expectErrorMessage("Invalid input arguments. 
Expected signatures are:\nf(STRING)\nf(INT)")
                );
        }
 

Reply via email to