dawidwys commented on a change in pull request #11280: [FLINK-16377][table] Support inline user defined functions in expression dsl URL: https://github.com/apache/flink/pull/11280#discussion_r397719901
########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/UserDefinedFunctionConvertRule.java ########## @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.expressions.converter; + +import org.apache.flink.table.expressions.ApiExpressionUtils; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; + +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * A call expression converter rule that converts calls to user defined functions. + */ +public class UserDefinedFunctionConvertRule implements CallExpressionConvertRule { + @Override + public Optional<RexNode> convert( + CallExpression call, + ConvertContext context) { + if (!(call.getFunctionDefinition() instanceof UserDefinedFunction)) { + return Optional.empty(); + } + + switch (call.getFunctionDefinition().getKind()) { + case SCALAR: + case TABLE: + List<RexNode> args = call.getChildren().stream().map(context::toRexNode).collect(Collectors.toList()); + return Optional.of(context.getRelBuilder().call( + BridgingSqlFunction.of( + context.getDataTypeFactory(), + context.getTypeFactory(), + SqlKind.OTHER_FUNCTION, + ApiExpressionUtils.getFunctionIdentifier(call), Review comment: No it cannot be done when constructing the call. The call is properly constructed. If the call is an inline call we do not have any function identifier and imo it is correct. It clearly identifies all the categories of calls: When function identifier is 1. null -> inline function 2. three part identifier -> catalog function 3. simple name/one part -> system function Imo the problem might be in the way `BridgingSqlFunction` handles identifiers/names. It requires a non null identifier to bridge it to calcite. Another option I see is we can change the way BridgingSqlFunction handles it. We can make the `SqlIdentifier` optional. And then move the logic from `ApiExpressionUtils#getInlineFunctionIdentifier` into the `BridgingSqlFunction`. What I have in mind is something along this: ``` final class BridgingUtils { static String createName(FunctionIdentifier identifier) { if (identifier.getSimpleName().isPresent()) { return identifier.getSimpleName().get(); } return identifier.getIdentifier() .map(ObjectIdentifier::getObjectName) .orElseThrow(IllegalStateException::new); } static SqlIdentifier createSqlIdentifier(FunctionIdentifier identifier) { return identifier.getIdentifier() .map(i -> new SqlIdentifier(i.toList(), SqlParserPos.ZERO)) .orElseGet(() -> new SqlIdentifier(identifier.getSimpleName().get(), SqlParserPos.ZERO)); // null indicates a built-in system function } } class BridgingSqlFunction extends SqlFunction { private BridgingSqlFunction( DataTypeFactory dataTypeFactory, FlinkTypeFactory typeFactory, SqlKind kind, FunctionIdentifier identifier, FunctionDefinition definition, TypeInference typeInference) { super( identifier != null ? createName(identifier) : createInlineFunctionName(definition), identifier != null ? createSqlIdentifier(identifier) : null, kind, createSqlReturnTypeInference(dataTypeFactory, definition, typeInference), createSqlOperandTypeInference(dataTypeFactory, definition, typeInference), createSqlOperandTypeChecker(dataTypeFactory, definition, typeInference), createParamTypes(typeFactory, typeInference), createSqlFunctionCategory(identifier)); this.dataTypeFactory = dataTypeFactory; this.typeFactory = typeFactory; this.identifier = identifier; this.definition = definition; this.typeInference = typeInference; } private static String createInlineFunctionName(FunctionDefinition functionDefinition) { if (functionDefinition instanceof BuiltInFunctionDefinition) { return ((BuiltInFunctionDefinition) functionDefinition).getName(); } else { final Optional<UserDefinedFunction> userDefinedFunction = extractUserDefinedFunction(functionDefinition); return userDefinedFunction.map(UserDefinedFunction::functionIdentifier) .orElseThrow(() -> new TableException("Unknown function type. Function: " + functionDefinition)); } } private static Optional<UserDefinedFunction> extractUserDefinedFunction(FunctionDefinition functionDefinition) { if (functionDefinition instanceof UserDefinedFunction) { return Optional.of((UserDefinedFunction) functionDefinition); } else if (functionDefinition instanceof ScalarFunctionDefinition) { return Optional.ofNullable(((ScalarFunctionDefinition) functionDefinition).getScalarFunction()); } else if (functionDefinition instanceof AggregateFunctionDefinition) { return Optional.ofNullable(((AggregateFunctionDefinition) functionDefinition).getAggregateFunction()); } else if (functionDefinition instanceof TableFunctionDefinition) { return Optional.ofNullable(((TableFunctionDefinition) functionDefinition).getTableFunction()); } else if (functionDefinition instanceof TableAggregateFunctionDefinition) { return Optional.ofNullable(((TableAggregateFunctionDefinition) functionDefinition).getTableAggregateFunction()); } return Optional.empty(); } } ``` This has the downside that, at least from Calcite's javadocs perspective, our inline functions would be Calcite's built-in functions. Nevertheless from our perspective I think this would be the cleanest approach. We can also do a mixed approach, more similar in the effect to the current state: ``` super( identifier != null ? createName(identifier) : createName(createInlineFunctionName(definition)), identifier != null ? createSqlIdentifier(identifier) : new SqlIdentifier(createInlineFunctionName(definition), SqlParserPos.ZERO), ... ) ``` ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
