slinkydeveloper commented on a change in pull request #17522:
URL: https://github.com/apache/flink/pull/17522#discussion_r737488635



##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractCodeGeneratorCastRule.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.utils.CastExecutor;
+import org.apache.flink.table.planner.codegen.CodeGenUtils;
+import org.apache.flink.table.planner.functions.casting.CastCodeBlock;
+import org.apache.flink.table.planner.functions.casting.CastRule;
+import org.apache.flink.table.planner.functions.casting.CastRulePredicate;
+import org.apache.flink.table.planner.functions.casting.CodeGeneratorCastRule;
+import org.apache.flink.table.runtime.generated.CompileUtils;
+import org.apache.flink.table.runtime.typeutils.InternalSerializers;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Base class for {@link CastRule} supporting code generation. This base class 
implements {@link
+ * #create(CastRule.Context, LogicalType, LogicalType)} compiling the 
generated code block into a
+ * {@link CastExecutor} implementation.
+ *
+ * <p>We suggest implementing {@link CodeGeneratorCastRule} starting from 
{@link
+ * AbstractNullAwareCodeGeneratorCastRule}, which provides nullability checks, 
or from {@link
+ * AbstractExpressionCodeGeneratorCastRule} to generate simple expression 
casts.
+ */
+public abstract class AbstractCodeGeneratorCastRule<IN, OUT> extends 
AbstractCastRule<IN, OUT>
+        implements CodeGeneratorCastRule<IN, OUT> {
+
+    protected AbstractCodeGeneratorCastRule(CastRulePredicate predicate) {
+        super(predicate);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public CastExecutor<IN, OUT> create(
+            CastRule.Context castRuleContext,
+            LogicalType inputLogicalType,
+            LogicalType targetLogicalType) {
+        final String inputTerm = "_myInput";
+        final String className = CodeGenUtils.newName("GeneratedCastExecutor");
+        final String inputTypeTerm = 
CodeGenUtils.boxedTypeTermForType(inputLogicalType);
+        final String targetTypeTerm = 
CodeGenUtils.boxedTypeTermForType(targetLogicalType);
+
+        final CastExecutorCodeGeneratorContext ctx =
+                new CastExecutorCodeGeneratorContext(castRuleContext);
+        final CastCodeBlock codeBlock =
+                generateCodeBlock(ctx, inputTerm, inputLogicalType, 
targetLogicalType);
+
+        // Class fields can contain type serializers
+        final String classFieldDecls =
+                ctx.getDeclaredTypeSerializers().stream()
+                        .map(name -> "private final TypeSerializer " + name + 
";\n")
+                        .collect(Collectors.joining());
+
+        final String constructorSignature =
+                "public "
+                        + className
+                        + "("
+                        + ctx.getDeclaredTypeSerializers().stream()
+                                .map(name -> "TypeSerializer " + name)
+                                .collect(Collectors.joining(", "))
+                        + ")";
+        final String constructorBody =
+                ctx.getDeclaredTypeSerializers().stream()
+                        .map(name -> "this." + name + " = " + name + ";\n")
+                        .collect(Collectors.joining());
+
+        // Because janino doesn't support generics, we need to manually cast 
the input variable of
+        // the cast method
+        final String functionSignature =
+                "@Override public Object cast(Object _myInputObj) throws "
+                        + CodeGenUtils.className(TableException.class);
+        final String inputVarDecl =
+                inputTypeTerm + " " + inputTerm + " = (" + inputTypeTerm + ") 
_myInputObj;\n";
+
+        final String returnStmt = "return " + codeBlock.getReturnTerm() + 
";\n";
+
+        final String classCode =
+                "public final class "
+                        + className
+                        + " implements "
+                        + CodeGenUtils.className(CastExecutor.class)
+                        + " {\n"
+                        + classFieldDecls
+                        + constructorSignature
+                        + " {\n"
+                        + constructorBody
+                        + "}\n"
+                        + functionSignature
+                        + "{\n"
+                        + inputVarDecl
+                        + String.join("\n", ctx.variableDeclarationStatements)
+                        + codeBlock.getCode()
+                        + "\n"
+                        + returnStmt
+                        + "}\n}";
+
+        try {
+            Object[] constructorArgs =
+                    ctx.getTypeSerializersInstances().toArray(new 
TypeSerializer[0]);
+            return (CastExecutor<IN, OUT>)
+                    CompileUtils.compile(
+                                    
Thread.currentThread().getContextClassLoader(),

Review comment:
       I've added it to the context, but i'm not sure in the codegen where to 
get the user classloader from. Shouldn't it be in `TableConfig`?




-- 
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


Reply via email to