You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Terry Wang (Jira)" <ji...@apache.org> on 2020/03/04 05:54:00 UTC

[jira] [Created] (FLINK-16414) create udaf/udtf function using sql casuing ValidationException: SQL validation failed. null

Terry Wang created FLINK-16414:
----------------------------------

             Summary: create udaf/udtf function using sql casuing ValidationException: SQL validation failed. null
                 Key: FLINK-16414
                 URL: https://issues.apache.org/jira/browse/FLINK-16414
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / API
    Affects Versions: 1.10.0
            Reporter: Terry Wang


When using TableEnvironment.sqlupdate() to create a udaf or udtf function, if the function doesn't override the getResultType() method, it's normal. But when using this function in the insert sql,  some exception like following will be throwed:

Exception in thread "main" org.apache.flink.table.api.ValidationException: SQL validation failed. null
	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:130)
	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:105)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:127)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlInsert(SqlToOperationConverter.java:342)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:142)
	at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:66)
	at org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlUpdate(TableEnvironmentImpl.java:484)

The reason is in FunctionDefinitionUtil#createFunctionDefinition, we shouldn't direct call t.getResultType or a.getAccumulatorType() or a.getResultType() but using UserDefinedFunctionHelper#getReturnTypeOfTableFunction
 UserDefinedFunctionHelper#getAccumulatorTypeOfAggregateFunction 
UserDefinedFunctionHelper#getReturnTypeOfAggregateFunction instead.
```

		if (udf instanceof ScalarFunction) {
			return new ScalarFunctionDefinition(
				name,
				(ScalarFunction) udf
			);
		} else if (udf instanceof TableFunction) {
			TableFunction t = (TableFunction) udf;
			return new TableFunctionDefinition(
				name,
				t,
				t.getResultType()
			);
		} else if (udf instanceof AggregateFunction) {
			AggregateFunction a = (AggregateFunction) udf;

			return new AggregateFunctionDefinition(
				name,
				a,
				a.getAccumulatorType(),
				a.getResultType()
			);
		} else if (udf instanceof TableAggregateFunction) {
			TableAggregateFunction a = (TableAggregateFunction) udf;

			return new TableAggregateFunctionDefinition(
				name,
				a,
				a.getAccumulatorType(),
				a.getResultType()
			);
```






--
This message was sent by Atlassian Jira
(v8.3.4#803005)