dawidwys commented on code in PR #23520: URL: https://github.com/apache/flink/pull/23520#discussion_r1358098238
########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeInferenceUtil.java: ########## @@ -385,7 +385,7 @@ private static String formatArgument(Signature.Argument arg) { return stringBuilder.toString(); } - private static boolean validateArgumentCount( + public static boolean validateArgumentCount( Review Comment: Please move it to the public methods section and add a javadoc. ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java: ########## @@ -62,11 +62,31 @@ public TypeInferenceOperandInference( this.typeInference = typeInference; } + private void validateArgumentCount(CallContext callContext) { + try { + TypeInferenceUtil.validateArgumentCount( + typeInference.getInputTypeStrategy().getArgumentCount(), + callContext.getArgumentDataTypes().size(), + true); + } catch (ValidationException e) { + final String msg = + String.format( + "%s\nExpected signatures are:\n%s", + e.getMessage(), + TypeInferenceUtil.generateSignature( + typeInference, + callContext.getName(), + callContext.getFunctionDefinition())); + throw new ValidationException(msg); + } + } + @Override public void inferOperandTypes( SqlCallBinding callBinding, RelDataType returnType, RelDataType[] operandTypes) { final CallContext callContext = new CallBindingCallContext(dataTypeFactory, definition, callBinding, returnType); + validateArgumentCount(callContext); try { inferOperandTypesOrError(unwrapTypeFactory(callBinding), callContext, operandTypes); } catch (ValidationException | CalciteContextException e) { Review Comment: I understand you'd like to improve the error message, but we should do it at the right location. I am sure that's not the right one. The contract for `TypeInferenceOperandInference` is that if we can not infer types, we should not change the types given. Otherwise, if we violate the contract we can not tell when and where this bites us back. If you want to improve the error message, you should change it where the exception is generated which is `SqlValidatorImpl`. I believe if we change `SqlValidatorImpl#handleUnresolvedFunction` to: ``` @Override public CalciteException handleUnresolvedFunction( SqlCall call, SqlOperator unresolvedFunction, List<RelDataType> argTypes, @Nullable List<String> argNames) { // For builtins, we can give a better error message final List<SqlOperator> overloads = new ArrayList<>(); opTab.lookupOperatorOverloads( unresolvedFunction.getNameAsId(), null, SqlSyntax.FUNCTION, overloads, catalogReader.nameMatcher()); if (overloads.size() == 1) { SqlFunction fun = (SqlFunction) overloads.get(0); if ((fun.getSqlIdentifier() == null) && (fun.getSyntax() != SqlSyntax.FUNCTION_ID)) { final int expectedArgCount = fun.getOperandCountRange().getMin(); throw newValidationError( call, RESOURCE.invalidArgCount(call.getOperator().getName(), expectedArgCount)); } } if (unresolvedFunction instanceof SqlFunction) { final SqlOperandTypeChecker typeChecking = new AssignableOperandTypeChecker(argTypes, argNames); final String invocation = typeChecking.getAllowedSignatures( unresolvedFunction, unresolvedFunction.getName()); if (unresolvedFunction.getOperandTypeChecker() != null) { final String signature = unresolvedFunction.getOperandTypeChecker() .getAllowedSignatures(unresolvedFunction, unresolvedFunction.getName()); throw newValidationError(call, EXTRA_RESOURCE.validatorNoFunctionMatch(invocation, signature)); // a new error message which takes invocation and supported signatures } else { throw newValidationError(call, RESOURCE.validatorUnknownFunction(invocation)); } } throw newValidationError(call, RESOURCE.validatorUnknownFunction(unresolvedFunction.getName())); } ``` you can achieve the improved error message you're looking for. I strongly believe we should upstream that to Calcite. Until we do contribute that to Calcite. We can add the `validatorNoFunctionMatch` to `ExtraCalciteResource` interface: ``` package org.apache.calcite.sql.validate; import org.apache.calcite.runtime.Resources; public interface ExtraCalciteResource { @Resources.BaseMessage("No match found for function signature {0}. Supported signatures: \n{1}") Resources.ExInst<SqlValidatorException> validatorNoFunctionMatch(String invocation, String allowedSignatures); } ``` and then instantiate that in `SqlValidatorImpl`: ``` private static final ExtraCalciteResource EXTRA_RESOURCE = Resources.create(ExtraCalciteResource.class); ``` ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java: ########## @@ -62,11 +62,31 @@ public TypeInferenceOperandInference( this.typeInference = typeInference; } + private void validateArgumentCount(CallContext callContext) { + try { + TypeInferenceUtil.validateArgumentCount( + typeInference.getInputTypeStrategy().getArgumentCount(), + callContext.getArgumentDataTypes().size(), + true); + } catch (ValidationException e) { + final String msg = + String.format( + "%s\nExpected signatures are:\n%s", + e.getMessage(), + TypeInferenceUtil.generateSignature( + typeInference, + callContext.getName(), + callContext.getFunctionDefinition())); + throw new ValidationException(msg); + } + } + @Override public void inferOperandTypes( SqlCallBinding callBinding, RelDataType returnType, RelDataType[] operandTypes) { final CallContext callContext = new CallBindingCallContext(dataTypeFactory, definition, callBinding, returnType); + validateArgumentCount(callContext); try { inferOperandTypesOrError(unwrapTypeFactory(callBinding), callContext, operandTypes); } catch (ValidationException | CalciteContextException e) { Review Comment: I don't think we should throw an exception here. I'd rather do: ```suggestion validateArgumentCount(callContext); try { if (TypeInferenceUtil.validateArgumentCount( typeInference.getInputTypeStrategy().getArgumentCount(), callContext.getArgumentDataTypes().size(), false)) { inferOperandTypesOrError(unwrapTypeFactory(callBinding), callContext, operandTypes); } } catch (ValidationException | CalciteContextException e) { ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org