dawidwys commented on a change in pull request #9485: [FLINK-13775][table-planner-blink] Refactor ExpressionConverter(RexNodeConverter) in blink URL: https://github.com/apache/flink/pull/9485#discussion_r319384061
########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CustomizedConvertRule.java ########## @@ -0,0 +1,389 @@ +/* + * 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.api.TableException; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionUtils; +import org.apache.flink.table.expressions.TableReferenceExpression; +import org.apache.flink.table.expressions.TypeLiteralExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.expressions.utils.ApiExpressionUtils; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.planner.calcite.FlinkRelBuilder; +import org.apache.flink.table.planner.functions.InternalFunctionDefinitions; +import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable; +import org.apache.flink.table.planner.functions.sql.SqlThrowExceptionFunction; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Preconditions; + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexSubQuery; +import org.apache.calcite.sql.fun.SqlTrimFunction; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.calcite.sql.type.SqlTypeName.VARCHAR; +import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.toLogicalType; +import static org.apache.flink.table.planner.expressions.converter.ExpressionConverter.extractValue; +import static org.apache.flink.table.planner.expressions.converter.ExpressionConverter.toRexNodes; +import static org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType; +import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isCharacterString; +import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isTemporal; +import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isTimeInterval; + +/** + * Customized {@link CallExpressionConvertRule}, Functions conversion here all require special logic, + * and there may be some special rules, such as needing get the literal values of inputs, such as + * converting to combinations of functions, to convert to RexNode of calcite. + */ +public class CustomizedConvertRule implements CallExpressionConvertRule { + + private static final Map<FunctionDefinition, Conversion> DEFINITION_RULE_MAP = new HashMap<>(); + static { + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.CAST, CustomizedConvertRule::convertCast); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.REINTERPRET_CAST, CustomizedConvertRule::convertReinterpretCast); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.IN, CustomizedConvertRule::convertIn); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.GET, CustomizedConvertRule::convertGet); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.TRIM, CustomizedConvertRule::convertTrim); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.AS, CustomizedConvertRule::convertAs); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.BETWEEN, CustomizedConvertRule::convertBetween); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.NOT_BETWEEN, CustomizedConvertRule::convertNotBetween); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.REPLACE, CustomizedConvertRule::convertReplace); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.PLUS, CustomizedConvertRule::convertPlus); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.CEIL, CustomizedConvertRule::convertCeil); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.FLOOR, CustomizedConvertRule::convertFloor); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS, CustomizedConvertRule::convertTemporalOverlaps); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.TIMESTAMP_DIFF, CustomizedConvertRule::convertTimestampDiff); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.ARRAY, CustomizedConvertRule::convertArray); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.MAP, CustomizedConvertRule::convertMap); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.ROW, CustomizedConvertRule::convertRow); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.ORDER_ASC, CustomizedConvertRule::convertOrderAsc); + DEFINITION_RULE_MAP.put(BuiltInFunctionDefinitions.SQRT, CustomizedConvertRule::convertSqrt); + + // blink expression + DEFINITION_RULE_MAP.put(InternalFunctionDefinitions.THROW_EXCEPTION, CustomizedConvertRule::convertThrowException); + } + + @Override + public Optional<RexNode> convert(CallExpression call, ConvertContext context) { + Conversion conversion = DEFINITION_RULE_MAP.get(call.getFunctionDefinition()); + return Optional.ofNullable(conversion).map(c -> c.convert(call, context)); + } + + private static RexNode convertCast(CallExpression call, ConvertContext context) { + checkArgumentNumber(call, 2); + RexNode child = context.toRexNode(call.getChildren().get(0)); + TypeLiteralExpression type = (TypeLiteralExpression) call.getChildren().get(1); + return context.getRelBuilder().getRexBuilder().makeAbstractCast( + context.getTypeFactory().createFieldTypeFromLogicalType( + type.getOutputDataType().getLogicalType().copy(child.getType().isNullable())), + child); + } + + private static RexNode convertOrderAsc(CallExpression call, ConvertContext context) { + checkArgumentNumber(call, 1); + return context.toRexNode(call.getChildren().get(0)); + } + + private static RexNode convertTimestampDiff(CallExpression call, ConvertContext context) { + checkArgumentNumber(call, 3); + List<RexNode> childrenRexNode = toRexNodes(context, call.getChildren()); + // different orders between flink table api and calcite. + return context.getRelBuilder().call(FlinkSqlOperatorTable.TIMESTAMP_DIFF, childrenRexNode.get(0), childrenRexNode.get(2), + childrenRexNode.get(1)); + } + + private static RexNode convertNotBetween(CallExpression call, ConvertContext context) { + checkArgumentNumber(call, 3); + List<RexNode> childrenRexNode = toRexNodes(context, call.getChildren()); + RexNode expr = childrenRexNode.get(0); + RexNode lowerBound = childrenRexNode.get(1); + RexNode upperBound = childrenRexNode.get(2); + return context.getRelBuilder().or( + context.getRelBuilder().call(FlinkSqlOperatorTable.LESS_THAN, expr, lowerBound), + context.getRelBuilder().call(FlinkSqlOperatorTable.GREATER_THAN, expr, upperBound)); + } + + private static RexNode convertBetween(CallExpression call, ConvertContext context) { + checkArgumentNumber(call, 3); + List<RexNode> childrenRexNode = toRexNodes(context, call.getChildren()); + RexNode expr = childrenRexNode.get(0); + RexNode lowerBound = childrenRexNode.get(1); + RexNode upperBound = childrenRexNode.get(2); + return context.getRelBuilder().and( + context.getRelBuilder().call(FlinkSqlOperatorTable.GREATER_THAN_OR_EQUAL, expr, lowerBound), + context.getRelBuilder().call(FlinkSqlOperatorTable.LESS_THAN_OR_EQUAL, expr, upperBound)); + } + + private static RexNode convertCeil(CallExpression call, ConvertContext context) { + checkArgumentNumber(call, 1, 2); + List<Expression> children = call.getChildren(); + List<RexNode> childrenRexNode = toRexNodes(context, children); + if (children.size() == 1) { + return context.getRelBuilder().call(FlinkSqlOperatorTable.CEIL, childrenRexNode); + } else { + return context.getRelBuilder().call(FlinkSqlOperatorTable.CEIL, childrenRexNode.get(1), childrenRexNode.get(0)); + } + } + + private static RexNode convertFloor(CallExpression call, ConvertContext context) { + checkArgumentNumber(call, 1, 2); + List<Expression> children = call.getChildren(); + List<RexNode> childrenRexNode = toRexNodes(context, children); + if (children.size() == 1) { + return context.getRelBuilder().call(FlinkSqlOperatorTable.FLOOR, childrenRexNode); + } else { + return context.getRelBuilder().call(FlinkSqlOperatorTable.FLOOR, childrenRexNode.get(1), childrenRexNode.get(0)); + } + } + + private static RexNode convertArray(CallExpression call, ConvertContext context) { + // TODO get type from CallExpression directly until introduce type inference on Expression + List<RexNode> childrenRexNode = toRexNodes(context, call.getChildren()); + ArrayType arrayType = new ArrayType(toLogicalType(childrenRexNode.get(0).getType())); + RelDataType relDataType = context.getTypeFactory().createFieldTypeFromLogicalType(arrayType); + return context.getRelBuilder().getRexBuilder().makeCall(relDataType, FlinkSqlOperatorTable.ARRAY_VALUE_CONSTRUCTOR, childrenRexNode); + } + + private static RexNode convertMap(CallExpression call, ConvertContext context) { + // TODO get type from CallExpression directly until introduce type inference on Expression + List<Expression> children = call.getChildren(); + Preconditions.checkArgument(!children.isEmpty() && children.size() % 2 == 0); Review comment: Shouldn't this be just `checkArgument`? ---------------------------------------------------------------- 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
