twalthr commented on a change in pull request #17738: URL: https://github.com/apache/flink/pull/17738#discussion_r748048173
########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ExpressionCodeGeneratorCastRule.java ########## @@ -0,0 +1,44 @@ +/* + * 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.functions.casting; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.types.logical.LogicalType; + +/** + * Cast rule that is able to generate a single expression containing all the casting logic. + * + * @param <IN> Input internal type + * @param <OUT> Output internal type + */ +@Internal +public interface ExpressionCodeGeneratorCastRule<IN, OUT> extends CodeGeneratorCastRule<IN, OUT> { + + /** + * Generate a Java expression performing the casting. This expression can be wrapped in another + * expression, or assigned to a variable or returned from a function. + * + * <p>NOTE: the {@code inputTerm} is always either a primitive or a non-null object. + */ + String generateExpression( Review comment: add JavaDoc for the return type? also either a primitive or a non-null object? ########## File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala ########## @@ -195,6 +195,38 @@ object CodeGenUtils { case _ => boxedTypeTermForType(t) } + /** + * Execute primitive unboxing. + */ + def unbox(term: String, ty: LogicalType): String = ty.getTypeRoot match { Review comment: nit: why in Scala? We could have moved it to CastUtils where also `CastRuleUtils.ternaryOperator` is located? ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/NumericPrimitiveToDecimalCastRule.java ########## @@ -0,0 +1,68 @@ +/* + * 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.functions.casting.rules; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.DecimalDataUtils; +import org.apache.flink.table.planner.functions.casting.CastRulePredicate; +import org.apache.flink.table.planner.functions.casting.CodeGeneratorCastRule; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import static org.apache.flink.table.planner.codegen.CodeGenUtils.className; +import static org.apache.flink.table.planner.functions.casting.rules.CastRuleUtils.methodCall; + +/** + * {@link LogicalTypeFamily#INTEGER_NUMERIC} and {@link LogicalTypeFamily#APPROXIMATE_NUMERIC} to + * {@link LogicalTypeRoot#DECIMAL} cast rule. + */ +@Internal +public class NumericPrimitiveToDecimalCastRule + extends AbstractExpressionCodeGeneratorCastRule<Number, DecimalData> { + + public static final NumericPrimitiveToDecimalCastRule INSTANCE = + new NumericPrimitiveToDecimalCastRule(); + + private NumericPrimitiveToDecimalCastRule() { + super( + CastRulePredicate.builder() + .input(LogicalTypeFamily.INTEGER_NUMERIC) + .input(LogicalTypeFamily.APPROXIMATE_NUMERIC) + .target(LogicalTypeRoot.DECIMAL) + .build()); + } + + @Override + public String generateExpression( + CodeGeneratorCastRule.Context context, + String inputTerm, + LogicalType inputLogicalType, + LogicalType targetLogicalType) { + final DecimalType targetDecimalType = (DecimalType) targetLogicalType; + return methodCall( + className(DecimalDataUtils.class), + "castFrom", Review comment: add this to `BuiltInMethods` ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractNullAwareCodeGeneratorCastRule.java ########## @@ -59,8 +60,8 @@ public CastCodeBlock generateCodeBlock( LogicalType targetType) { final CastRuleUtils.CodeWriter writer = new CastRuleUtils.CodeWriter(); - // Result of a casting can be null only and only if the input is null - final boolean isResultNullable = inputType.isNullable(); + // Result of a casting is null if the input is null or the output primitive type is nullable Review comment: nit: is this comment helpful? it basically describes the actually very readable expression below. ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/NumericPrimitiveCastRule.java ########## @@ -0,0 +1,79 @@ +/* + * 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.functions.casting.rules; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.planner.functions.casting.CastRulePredicate; +import org.apache.flink.table.planner.functions.casting.CodeGeneratorCastRule; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; + +import static org.apache.flink.table.types.logical.LogicalTypeFamily.APPROXIMATE_NUMERIC; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.INTEGER_NUMERIC; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.BIGINT; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTEGER; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_DAY_TIME; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_YEAR_MONTH; + +/** + * Cast rule for {@link LogicalTypeFamily#INTEGER_NUMERIC} and {@link + * LogicalTypeFamily#APPROXIMATE_NUMERIC} and {@link LogicalTypeFamily#INTERVAL} conversions. + */ +@Internal +public class NumericPrimitiveCastRule + extends AbstractExpressionCodeGeneratorCastRule<Number, Number> { + + public static final NumericPrimitiveCastRule INSTANCE = new NumericPrimitiveCastRule(); + + private NumericPrimitiveCastRule() { + super(CastRulePredicate.builder().predicate(NumericPrimitiveCastRule::matches).build()); + } + + private static boolean matches(LogicalType x, LogicalType y) { Review comment: `src` /`dest` or something meaningful? ########## File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java ########## @@ -236,6 +471,35 @@ RawValueData.fromObject( LocalDateTime.parse("2020-11-11T18:08:01.123")), StringData.fromString("2020-11-11T18:08:01.123")), + CastTestSpecBuilder.testCastTo(DECIMAL(5, 3)) + .fromCase( + DECIMAL(4, 3), + DecimalData.fromBigDecimal(new BigDecimal("9.87"), 4, 3), + DecimalData.fromBigDecimal(new BigDecimal("9.870"), 5, 3)) + .fromCase( + TINYINT(), + (byte) -1, + DecimalData.fromBigDecimal(new BigDecimal("-1.000"), 5, 3)) Review comment: introduce a help method for the very repetitive `DecimalData.fromBigDecimal(new BigDecimal("-1.000"), 5, 3)`? ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/DecimalToNumericPrimitiveCastRule.java ########## @@ -0,0 +1,68 @@ +/* + * 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.functions.casting.rules; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.planner.functions.casting.CastRulePredicate; +import org.apache.flink.table.planner.functions.casting.CodeGeneratorCastRule; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import java.lang.reflect.Method; + +import static org.apache.flink.table.planner.codegen.calls.BuiltInMethods.DECIMAL_TO_DOUBLE; +import static org.apache.flink.table.planner.codegen.calls.BuiltInMethods.DECIMAL_TO_INTEGRAL; +import static org.apache.flink.table.planner.functions.casting.rules.CastRuleUtils.castToPrimitive; +import static org.apache.flink.table.planner.functions.casting.rules.CastRuleUtils.staticCall; + +/** + * {@link LogicalTypeRoot#DECIMAL} to {@link LogicalTypeFamily#INTEGER_NUMERIC} and {@link + * LogicalTypeFamily#APPROXIMATE_NUMERIC} cast rule. + */ +@Internal +public class DecimalToNumericPrimitiveCastRule Review comment: can we move the rules one package level up? we do the same for `DataStructureConverter`. This allows to have all rules package private and we don't spam the classpath. And having everything under `casting` should be abstraction enough. -- 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]
