twalthr commented on a change in pull request #17522: URL: https://github.com/apache/flink/pull/17522#discussion_r734267465
########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastExpression.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.functions.casting; + +import org.apache.flink.table.types.logical.LogicalType; + +import java.util.Objects; + +/** + * Generated cast expression result. This POJO contains the Java code of the expression and the + * returned type. + */ +public class CastExpression { Review comment: annotate all added classes with `@Internal` or `@PublicEvolving` ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratedCastExecutor.java ########## @@ -0,0 +1,57 @@ +/* + * 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.table.api.TableException; +import org.apache.flink.table.data.utils.CastExecutor; +import org.apache.flink.util.FlinkRuntimeException; + +import org.codehaus.janino.ExpressionEvaluator; + +import java.lang.reflect.InvocationTargetException; + +/** + * Cast executor which can be instantiated starting from an expression code. + * + * @param <IN> Input internal type + * @param <OUT> Output internal type + */ +class CodeGeneratedCastExecutor<IN, OUT> implements CastExecutor<IN, OUT> { Review comment: Is this class `Serializable`? Usually, classes with code generation need to be serialized with a string and their `open()` method calls the compiler after shipping. This is also the case for `DataStructureConverter` e.g. structured types which also use code generation. We can also remove this interface for now. ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/data/casting/rules/UpcastToBigIntCastRule.java ########## @@ -0,0 +1,50 @@ +/* + * 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.data.casting.rules; + +import org.apache.flink.table.data.casting.CastExpression; +import org.apache.flink.table.data.casting.CastRulePredicate; +import org.apache.flink.table.data.casting.CodeGeneratorCastRule; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +/** Upcasting to long for smaller types. */ +public class UpcastToBigIntCastRule extends AbstractCodeGeneratorCastRule<Object, Long> { + + public static final UpcastToBigIntCastRule INSTANCE = new UpcastToBigIntCastRule(); + + protected UpcastToBigIntCastRule() { Review comment: why not private? ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java ########## @@ -0,0 +1,146 @@ +/* + * 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.planner.functions.casting.rules.IdentityCastRule; +import org.apache.flink.table.planner.functions.casting.rules.TimestampToStringCastRule; +import org.apache.flink.table.planner.functions.casting.rules.UpcastToBigIntCastRule; +import org.apache.flink.table.types.DataType; +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 javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** This class resolves {@link CastRule} using the input and the target type. */ +@Internal +public class CastRuleProvider { + + /* ------- Entrypoint ------- */ + + /** + * Resolve a {@link CastRule} for the provided input type and target type. Returns {@code null} + * if no rule can be resolved. + */ + public static @Nullable CastRule<?, ?> resolve(LogicalType inputType, LogicalType targetType) { Review comment: My comment that you have to pass the same arguments twice until you get an executor is still not addressed. ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/data/casting/CastRuleProvider.java ########## @@ -30,7 +30,6 @@ import javax.annotation.Nullable; Review comment: Regarding commit naming: This is not a `[hotfix]` commit. Hotfix commits will be visible in the log after merging. This commit is a fixup commit that the committer should squash into the previous one. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org