JingsongLi commented on a change in pull request #10960: [FLINK-15487][table] 
Update code generation for new type inference
URL: https://github.com/apache/flink/pull/10960#discussion_r373394902
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingSqlFunctionCallGen.scala
 ##########
 @@ -0,0 +1,196 @@
+/*
+ * 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.codegen.calls
+import java.lang.reflect.Method
+import java.util.Collections
+
+import org.apache.calcite.rex.{RexCall, RexCallBinding}
+import org.apache.flink.table.functions.UserDefinedFunctionHelper.SCALAR_EVAL
+import org.apache.flink.table.functions.{ScalarFunction, TableFunction, 
UserDefinedFunction, UserDefinedFunctionHelper}
+import 
org.apache.flink.table.planner.codegen.CodeGenUtils.{genToExternalIfNeeded, 
genToInternalIfNeeded, typeTerm}
+import org.apache.flink.table.planner.codegen.{CodeGenException, 
CodeGeneratorContext, GeneratedExpression}
+import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction
+import 
org.apache.flink.table.planner.functions.inference.OperatorBindingCallContext
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil.toScala
+import org.apache.flink.table.types.DataType
+import org.apache.flink.table.types.extraction.utils.ExtractionUtils
+import 
org.apache.flink.table.types.extraction.utils.ExtractionUtils.{createMethodSignatureString,
 isAssignable, isMethodInvokable, primitiveToWrapper}
+import org.apache.flink.table.types.inference.TypeInferenceUtil
+import org.apache.flink.table.types.logical.LogicalType
+
+/**
+ * Generates a call to a user-defined [[ScalarFunction]] or [[TableFunction]] 
(future work).
+ */
+class BridgingSqlFunctionCallGen(call: RexCall) extends CallGenerator {
+
+  private val function: BridgingSqlFunction = 
call.getOperator.asInstanceOf[BridgingSqlFunction]
+  private val udf: UserDefinedFunction = 
function.getDefinition.asInstanceOf[UserDefinedFunction]
+
+  override def generate(
+      ctx: CodeGeneratorContext,
+      operands: Seq[GeneratedExpression],
+      returnType: LogicalType)
+    : GeneratedExpression = {
+
+    val inference = function.getTypeInference
+
+    // we could have implemented a dedicated code generation context but the 
closer we are to
+    // Calcite the more consistent is the type inference during the data type 
enrichment
+    val callContext = new OperatorBindingCallContext(
+      function.getDataTypeFactory,
+      udf,
+      RexCallBinding.create(
+        function.getTypeFactory,
+        call,
+        Collections.emptyList()))
+
+    // enrich argument types with conversion class
+    val adaptedCallContext = TypeInferenceUtil.adaptArguments(
+      inference,
+      callContext,
+      null)
+    val enrichedArgumentDataTypes = 
toScala(adaptedCallContext.getArgumentDataTypes)
+    verifyArgumentTypes(operands.map(_.resultType), enrichedArgumentDataTypes)
+
+    // enrich output types with conversion class
+    val enrichedOutputDataType = TypeInferenceUtil.inferOutputType(
+      adaptedCallContext,
+      inference.getOutputTypeStrategy)
+    verifyOutputType(returnType, enrichedOutputDataType)
+
+    // find runtime method and generate call
+    verifyImplementation(enrichedArgumentDataTypes, enrichedOutputDataType)
+    generateFunctionCall(ctx, operands, enrichedArgumentDataTypes, 
enrichedOutputDataType)
+  }
+
+  private def generateFunctionCall(
+      ctx: CodeGeneratorContext,
+      operands: Seq[GeneratedExpression],
+      argumentDataTypes: Seq[DataType],
+      outputDataType: DataType)
+    : GeneratedExpression = {
+
+    val functionTerm = ctx.addReusableFunction(udf)
+
+    // operand conversion
+    val externalOperands = prepareExternalOperands(ctx, operands, 
argumentDataTypes)
+    val externalOperandTerms = externalOperands.map(_.resultTerm).mkString(", 
")
+
+    // result conversion
+    val externalResultClass = outputDataType.getConversionClass
+    val externalResultTypeTerm = typeTerm(externalResultClass)
+    // Janino does not fully support the JVM spec:
+    // boolean b = (boolean) f(); where f returns Object
+    // This is not supported and we need to box manually.
+    val externalResultClassBoxed = primitiveToWrapper(externalResultClass)
+    val externalResultCasting = if (externalResultClass == 
externalResultClassBoxed) {
+      s"($externalResultTypeTerm)"
+    } else {
+      s"($externalResultTypeTerm) (${typeTerm(externalResultClassBoxed)})"
 
 Review comment:
   Can we check the method return class too? If it return primitive class, we 
don't need add this cast.

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

Reply via email to