fsk119 commented on code in PR #24106:
URL: https://github.com/apache/flink/pull/24106#discussion_r1464215746
##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/ProcedureHint.java:
##########
@@ -139,6 +139,18 @@
*/
String[] argumentNames() default {""};
+ /**
+ * Explicitly lists the argument that a procedure takes as input,
including their names, types,
+ * and whether they are optional.
+ *
+ * <p>By default, it is recommended to use this parameter instead of
{@link #input()}. If the
+ * type of argumentHint is not defined, it will be considered an invalid
argument and an
+ * exception will be thrown. Additionally, both this parameter and {@link
#input()} cannot be
+ * defined at the same time. If neither arguments nor {@link #input()} are
defined,
+ * reflection-based extraction will be used.
+ */
+ ArgumentHint[] arguments() default {};
Review Comment:
ditto
##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java:
##########
@@ -362,7 +363,17 @@ static Optional<FunctionArgumentTemplate>
tryExtractInputGroupArgument(
Method method, int paramPos) {
final Parameter parameter = method.getParameters()[paramPos];
final DataTypeHint hint = parameter.getAnnotation(DataTypeHint.class);
- if (hint != null) {
+ final ArgumentHint argumentHint =
parameter.getAnnotation(ArgumentHint.class);
+ if (hint != null && argumentHint != null) {
+ throw extractionError(
+ "ArgumentHint and DataTypeHint cannot be declared at the
same time.");
Review Comment:
nit: it's better to notify user which parameter is not correct.
```
throw extractionError(
String.format(
"ArgumentHint and DataTypeHint cannot be
declared at the same time for parameter '%s'.",
parameter.getName()));
```
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java:
##########
@@ -114,6 +117,24 @@ public boolean isOptional(int i) {
return false;
}
+ @Override
+ public List<RelDataType> paramTypes(RelDataTypeFactory typeFactory) {
+ throw new IllegalStateException("Should not be called");
+ }
+
+ @Override
+ public List<String> paramNames() {
+ return typeInference
Review Comment:
Add more details exception. Print current required method method signature
and the candidate method signatures.
BTW, do you mean user can not use like this?
```
public static class NamedArgumentsScalarFunction extends ScalarFunction {
@FunctionHint(
output = @DataTypeHint("STRING"),
arguments = {
@ArgumentHint(name = "in1", type = @DataTypeHint("int")),
@ArgumentHint(name = "in2", type = @DataTypeHint("int"),
isOptional = true)
})
public String eval(Integer arg1, Integer arg2) {
return (arg1 + ": " + arg2);
}
@FunctionHint(
output = @DataTypeHint("STRING"),
arguments = {
@ArgumentHint(name = "in1", type =
@DataTypeHint("int")),
@ArgumentHint(name = "in2", type =
@DataTypeHint("int"), isOptional = true),
@ArgumentHint(name = "in3", type =
@DataTypeHint("int"), isOptional = true)
})
public String eval(Integer arg1, Integer arg2, Integer arg3) {
return (arg1 + ": " + arg2);
}
}
```
```
SELECT udf(in1 => 1);
```
##########
flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java:
##########
@@ -1585,4 +1629,106 @@ public void eval(CompletableFuture<Long> f, int[] i) {}
private static class DataTypeHintOnScalarFunctionAsync extends
AsyncScalarFunction {
public void eval(@DataTypeHint("ROW<i INT>")
CompletableFuture<RowData> f) {}
}
+
+ private static class ArgumentHintScalarFunction extends ScalarFunction {
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ })
+ public String eval(String f1, int f2) {
+ return "";
+ }
+ }
+
+ private static class ArgumentsAndInputsScalarFunction extends
ScalarFunction {
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ },
+ input = {@DataTypeHint("STRING"), @DataTypeHint("INTEGER")})
+ public String eval(String f1, int f2) {
+ return "";
+ }
+ }
+
+ private static class ArgumentsHintAndDataTypeHintScalarFunction extends
ScalarFunction {
+
+ public String eval(
+ @DataTypeHint("STRING") @ArgumentHint(name = "f1", type =
@DataTypeHint("STRING"))
+ String f1,
+ @ArgumentHint(name = "f2", type = @DataTypeHint("INTEGER"))
int f2) {
+ return "";
+ }
+ }
+
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ })
+ private static class InvalidFunctionHintOnClassAndMethod extends
ScalarFunction {
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ },
+ input = {@DataTypeHint("STRING"), @DataTypeHint("INTEGER")})
+ public String eval(String f1, int f2) {
+ return "";
+ }
+ }
+
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ })
+ private static class ValidFunctionHintOnClassAndMethod extends
ScalarFunction {
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ })
+ public String eval(String f1, int f2) {
+ return "";
+ }
+ }
+
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ })
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ })
+ private static class ScalarFunctionWithFunctionHintConflictMethod extends
ScalarFunction {
+ public String eval(String f1, int f2) {
+ return "";
+ }
+ }
+
+ private static class ArgumentsHintScalarFunctionWithOverloadedFunction
extends ScalarFunction {
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("INTEGER"), name = "f2")
+ })
+ public String eval(String f1, int f2) {
+ return "";
+ }
+
+ @FunctionHint(
+ arguments = {
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f1"),
+ @ArgumentHint(type = @DataTypeHint("STRING"), name = "f2")
+ })
+ public String eval(String f1, String f2) {
Review Comment:
Can user specify like this
```
public String eval(@ArgumentHint(type = @DataTypeHint("STRING"), name =
"f1") String f1) {}
```
##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/DataTypeTemplate.java:
##########
@@ -118,6 +119,10 @@ static DataTypeTemplate fromAnnotation(DataTypeFactory
typeFactory, DataTypeHint
* Creates an instance from the given {@link DataTypeHint} with a resolved
data type if
* available.
*/
Review Comment:
The javadoc is not for this method.
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java:
##########
@@ -114,6 +117,24 @@ public boolean isOptional(int i) {
return false;
}
+ @Override
+ public List<RelDataType> paramTypes(RelDataTypeFactory typeFactory) {
+ throw new IllegalStateException("Should not be called");
Review Comment:
Can you share some thoughts why this impossible?
##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/FunctionHint.java:
##########
@@ -148,6 +148,18 @@
*/
String[] argumentNames() default {""};
+ /**
+ * Explicitly lists the argument that a function takes as input, including
their names, types,
+ * and whether they are optional.
+ *
+ * <p>By default, it is recommended to use this parameter instead of
{@link #input()}. If the
+ * type of argumentHint is not defined, it will be considered an invalid
argument and an
+ * exception will be thrown. Additionally, both this parameter and {@link
#input()} cannot be
+ * defined at the same time. If neither arguments nor {@link #input()} are
defined,
+ * reflection-based extraction will be used.
+ */
+ ArgumentHint[] arguments() default {};
Review Comment:
Please also add some examples at line 46 including how to specify an
arugment is a variable-length argument.
BTW, it seems that it uses `input` rather than `inputs` in the annotation.
However, in the FLIP it uses `arguments` rather than `argument`.
##########
flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java:
##########
@@ -3730,7 +3733,11 @@ private void checkRollUp(
// can be another SqlCall, or an SqlIdentifier.
checkRollUp(grandParent, parent, stripDot, scope,
contextClause);
} else {
- List<? extends @Nullable SqlNode> children = ((SqlCall)
stripDot).getOperandList();
+ // ----- FLINK MODIFICATION BEGIN -----
+ SqlCall call = (SqlCall) stripDot;
+ List<? extends @Nullable SqlNode> children =
+ new SqlCallBinding(this, scope, call).operands();
Review Comment:
Could you explain why modify this? If this is a bug, can you open a ticket
on the calcite side, so we can track the progress.
##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/FunctionTemplate.java:
##########
@@ -173,18 +183,45 @@ private static <T, H extends Annotation> T defaultAsNull(
private static @Nullable FunctionSignatureTemplate createSignatureTemplate(
DataTypeFactory typeFactory,
- @Nullable DataTypeHint[] input,
+ @Nullable DataTypeHint[] inputs,
@Nullable String[] argumentNames,
+ @Nullable ArgumentHint[] argumentHints,
boolean isVarArg) {
- if (input == null) {
+
+ String[] argumentHintNames;
+ DataTypeHint[] argumentHintTypes;
+
+ if (argumentHints != null && inputs != null) {
+ throw extractionError(
+ "Unable to support specifying both inputs and arguments at
the same time.");
+ }
+
+ if (argumentHints != null) {
+ argumentHintNames =
+ Arrays.stream(argumentHints)
+ .map(argumentHint -> defaultAsNull(argumentHint,
ArgumentHint::name))
+ .filter(Objects::nonNull)
+ .toArray(String[]::new);
+ argumentHintTypes =
+ Arrays.stream(argumentHints)
+ .map(argumentHint -> defaultAsNull(argumentHint,
ArgumentHint::type))
+ .filter(Objects::nonNull)
+ .toArray(DataTypeHint[]::new);
+ } else {
+ argumentHintTypes = inputs;
+ argumentHintNames = argumentNames;
+ }
+
+ if (argumentHintTypes == null) {
Review Comment:
If user doesn't specify argument type, e.g.
```
public static class NamedArgumentsScalarFunction extends ScalarFunction {
@FunctionHint(
output = @DataTypeHint("STRING"),
arguments = {@ArgumentHint(name = "in1"), @ArgumentHint(name
= "in2")})
public String eval(Integer arg1, Integer arg2) {
return (arg1 + ": " + arg2);
}
}
```
can we still use named argument in the SQL?
##########
flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java:
##########
@@ -1585,4 +1629,106 @@ public void eval(CompletableFuture<Long> f, int[] i) {}
private static class DataTypeHintOnScalarFunctionAsync extends
AsyncScalarFunction {
public void eval(@DataTypeHint("ROW<i INT>")
CompletableFuture<RowData> f) {}
}
+
+ private static class ArgumentHintScalarFunction extends ScalarFunction {
Review Comment:
if udf's argument hints have the same names, the planner throw exception to
notify users?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]