lihaosky commented on code in PR #27041:
URL: https://github.com/apache/flink/pull/27041#discussion_r2430133026
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/FunctionCallContext.java:
##########
@@ -40,15 +40,15 @@
/** The {@link CallContext} of a {@link LookupTableSource} runtime function. */
@Internal
-public class LookupCallContext extends AbstractSqlCallContext {
+public class FunctionCallContext extends AbstractSqlCallContext {
private final List<FunctionParam> lookupKeys;
private final List<DataType> argumentDataTypes;
private final DataType outputDataType;
- public LookupCallContext(
+ public FunctionCallContext(
Review Comment:
Does the `lookupType` make sense anymore? Should it be `outputType`?
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCallCodeGenerator.scala:
##########
@@ -0,0 +1,344 @@
+/*
+ * 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
+
+import org.apache.flink.api.common.functions.{FlatMapFunction, Function,
OpenContext}
+import org.apache.flink.configuration.ReadableConfig
+import org.apache.flink.streaming.api.functions.async.AsyncFunction
+import org.apache.flink.table.catalog.DataTypeFactory
+import org.apache.flink.table.data.{GenericRowData, RowData}
+import org.apache.flink.table.data.utils.JoinedRowData
+import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction,
UserDefinedFunction, UserDefinedFunctionHelper}
+import
org.apache.flink.table.planner.codegen.CodeGenUtils.{boxedTypeTermForType,
className, newName, DEFAULT_COLLECTOR_TERM, DEFAULT_INPUT1_TERM,
DEFAULT_INPUT2_TERM}
+import
org.apache.flink.table.planner.codegen.GenerateUtils.{generateInputAccess,
generateLiteral}
+import org.apache.flink.table.planner.delegation.PlannerBase
+import org.apache.flink.table.planner.functions.inference.FunctionCallContext
+import org.apache.flink.table.planner.plan.utils.FunctionCallUtil.{Constant,
FieldRef, FunctionParam}
+import org.apache.flink.table.planner.plan.utils.RexLiteralUtil
+import org.apache.flink.table.runtime.collector.ListenableCollector
+import
org.apache.flink.table.runtime.collector.ListenableCollector.CollectListener
+import org.apache.flink.table.runtime.generated.{GeneratedCollector,
GeneratedFunction}
+import org.apache.flink.table.types.DataType
+import org.apache.flink.table.types.logical.{LogicalType, RowType}
+import org.apache.flink.util.Collector
+
+import org.apache.calcite.rex.RexNode
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+object FunctionCallCodeGenerator {
+
+ case class GeneratedTableFunctionWithDataType[F <: Function](
+ tableFunc: GeneratedFunction[F],
+ dataType: DataType)
+
+ /** Generates a sync function ([[TableFunction]]) call. */
+ def generateSyncFunctionCall(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ syncFunctionDefinition: TableFunction[_],
+ inferCall: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String,
+ fieldCopy: Boolean):
GeneratedTableFunctionWithDataType[FlatMapFunction[RowData, RowData]] = {
+
+ val bodyCode: GeneratedExpression => String = call => {
+ val resultCollectorTerm = call.resultTerm
+ s"""
+ |$resultCollectorTerm.setCollector($DEFAULT_COLLECTOR_TERM);
+ |${call.code}
+ |""".stripMargin
+ }
+
+ generateFunctionCall(
+ classOf[FlatMapFunction[RowData, RowData]],
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ functionOutputType,
+ collectorOutputType,
+ parameters,
+ syncFunctionDefinition,
+ inferCall,
+ functionName,
+ generateClassName,
+ fieldCopy,
+ bodyCode
+ )
+ }
+
+ /** Generates an async function ([[AsyncTableFunction]]) call. */
+ def generateAsyncFunctionCall(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ asyncFunctionDefinition: AsyncTableFunction[_],
+ generateCallWithDataType: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String
+ ): GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = {
+ generateFunctionCall(
+ classOf[AsyncFunction[RowData, AnyRef]],
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ functionOutputType,
+ collectorOutputType,
+ parameters,
+ asyncFunctionDefinition,
+ generateCallWithDataType,
+ functionName,
+ generateClassName,
+ fieldCopy = true,
+ _.code
+ )
+ }
+
+ private def generateFunctionCall[F <: Function](
+ generatedClass: Class[F],
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ functionDefinition: UserDefinedFunction,
+ generateCallWithDataType: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String,
+ fieldCopy: Boolean,
+ bodyCode: GeneratedExpression => String):
GeneratedTableFunctionWithDataType[F] = {
+
+ val callContext =
+ new FunctionCallContext(
+ dataTypeFactory,
+ functionDefinition,
+ inputType,
+ parameters,
+ functionOutputType)
+
+ // create the final UDF for runtime
+ val udf = UserDefinedFunctionHelper.createSpecializedFunction(
+ functionName,
+ functionDefinition,
+ callContext,
+ classOf[PlannerBase].getClassLoader,
+ tableConfig,
+ // no need to support expression evaluation at this point
+ null
+ )
+
+ val ctx = new CodeGeneratorContext(tableConfig, classLoader)
+ val operands = prepareOperands(ctx, inputType, parameters, fieldCopy)
+
+ val callWithDataType: (GeneratedExpression, DataType) =
+ generateCallWithDataType(ctx, callContext, udf, operands)
+
+ val function = FunctionCodeGenerator.generateFunction(
+ ctx,
+ generateClassName,
+ generatedClass,
+ bodyCode(callWithDataType._1),
+ collectorOutputType,
+ inputType)
+
+ GeneratedTableFunctionWithDataType(function, callWithDataType._2)
+ }
+
+ private def prepareOperands(
+ ctx: CodeGeneratorContext,
+ inputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ fieldCopy: Boolean): Seq[GeneratedExpression] = {
+
+ parameters.asScala
+ .map {
+ case constantKey: Constant =>
+ val res = RexLiteralUtil.toFlinkInternalValue(constantKey.literal)
+ generateLiteral(ctx, res.f0, res.f1)
+ case fieldKey: FieldRef =>
+ generateInputAccess(
+ ctx,
+ inputType,
+ DEFAULT_INPUT1_TERM,
+ fieldKey.index,
+ nullableInput = false,
+ fieldCopy)
+ case _ =>
+ throw new CodeGenException("Invalid lookup key.")
Review Comment:
Invalid parameters?
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/FunctionCallContext.java:
##########
@@ -40,15 +40,15 @@
/** The {@link CallContext} of a {@link LookupTableSource} runtime function. */
@Internal
-public class LookupCallContext extends AbstractSqlCallContext {
+public class FunctionCallContext extends AbstractSqlCallContext {
private final List<FunctionParam> lookupKeys;
Review Comment:
Does lookupKeys still make sense for new name `FunctionCallContext`? Should
this be `params` or something else?
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCallCodeGenerator.scala:
##########
@@ -0,0 +1,344 @@
+/*
+ * 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
+
+import org.apache.flink.api.common.functions.{FlatMapFunction, Function,
OpenContext}
+import org.apache.flink.configuration.ReadableConfig
+import org.apache.flink.streaming.api.functions.async.AsyncFunction
+import org.apache.flink.table.catalog.DataTypeFactory
+import org.apache.flink.table.data.{GenericRowData, RowData}
+import org.apache.flink.table.data.utils.JoinedRowData
+import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction,
UserDefinedFunction, UserDefinedFunctionHelper}
+import
org.apache.flink.table.planner.codegen.CodeGenUtils.{boxedTypeTermForType,
className, newName, DEFAULT_COLLECTOR_TERM, DEFAULT_INPUT1_TERM,
DEFAULT_INPUT2_TERM}
+import
org.apache.flink.table.planner.codegen.GenerateUtils.{generateInputAccess,
generateLiteral}
+import org.apache.flink.table.planner.delegation.PlannerBase
+import org.apache.flink.table.planner.functions.inference.FunctionCallContext
+import org.apache.flink.table.planner.plan.utils.FunctionCallUtil.{Constant,
FieldRef, FunctionParam}
+import org.apache.flink.table.planner.plan.utils.RexLiteralUtil
+import org.apache.flink.table.runtime.collector.ListenableCollector
+import
org.apache.flink.table.runtime.collector.ListenableCollector.CollectListener
+import org.apache.flink.table.runtime.generated.{GeneratedCollector,
GeneratedFunction}
+import org.apache.flink.table.types.DataType
+import org.apache.flink.table.types.logical.{LogicalType, RowType}
+import org.apache.flink.util.Collector
+
+import org.apache.calcite.rex.RexNode
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+object FunctionCallCodeGenerator {
+
+ case class GeneratedTableFunctionWithDataType[F <: Function](
+ tableFunc: GeneratedFunction[F],
+ dataType: DataType)
+
+ /** Generates a sync function ([[TableFunction]]) call. */
+ def generateSyncFunctionCall(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ syncFunctionDefinition: TableFunction[_],
+ inferCall: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String,
+ fieldCopy: Boolean):
GeneratedTableFunctionWithDataType[FlatMapFunction[RowData, RowData]] = {
+
+ val bodyCode: GeneratedExpression => String = call => {
+ val resultCollectorTerm = call.resultTerm
+ s"""
+ |$resultCollectorTerm.setCollector($DEFAULT_COLLECTOR_TERM);
+ |${call.code}
+ |""".stripMargin
+ }
+
+ generateFunctionCall(
+ classOf[FlatMapFunction[RowData, RowData]],
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ functionOutputType,
+ collectorOutputType,
+ parameters,
+ syncFunctionDefinition,
+ inferCall,
+ functionName,
+ generateClassName,
+ fieldCopy,
+ bodyCode
+ )
+ }
+
+ /** Generates an async function ([[AsyncTableFunction]]) call. */
+ def generateAsyncFunctionCall(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ asyncFunctionDefinition: AsyncTableFunction[_],
+ generateCallWithDataType: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String
+ ): GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = {
+ generateFunctionCall(
+ classOf[AsyncFunction[RowData, AnyRef]],
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ functionOutputType,
+ collectorOutputType,
+ parameters,
+ asyncFunctionDefinition,
+ generateCallWithDataType,
+ functionName,
+ generateClassName,
+ fieldCopy = true,
+ _.code
+ )
+ }
+
+ private def generateFunctionCall[F <: Function](
+ generatedClass: Class[F],
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ functionDefinition: UserDefinedFunction,
+ generateCallWithDataType: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String,
+ fieldCopy: Boolean,
+ bodyCode: GeneratedExpression => String):
GeneratedTableFunctionWithDataType[F] = {
+
+ val callContext =
+ new FunctionCallContext(
+ dataTypeFactory,
+ functionDefinition,
+ inputType,
+ parameters,
+ functionOutputType)
+
+ // create the final UDF for runtime
+ val udf = UserDefinedFunctionHelper.createSpecializedFunction(
+ functionName,
+ functionDefinition,
+ callContext,
+ classOf[PlannerBase].getClassLoader,
+ tableConfig,
+ // no need to support expression evaluation at this point
+ null
+ )
+
+ val ctx = new CodeGeneratorContext(tableConfig, classLoader)
+ val operands = prepareOperands(ctx, inputType, parameters, fieldCopy)
+
+ val callWithDataType: (GeneratedExpression, DataType) =
+ generateCallWithDataType(ctx, callContext, udf, operands)
+
+ val function = FunctionCodeGenerator.generateFunction(
+ ctx,
+ generateClassName,
+ generatedClass,
+ bodyCode(callWithDataType._1),
+ collectorOutputType,
+ inputType)
+
+ GeneratedTableFunctionWithDataType(function, callWithDataType._2)
+ }
+
+ private def prepareOperands(
+ ctx: CodeGeneratorContext,
+ inputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ fieldCopy: Boolean): Seq[GeneratedExpression] = {
+
+ parameters.asScala
+ .map {
+ case constantKey: Constant =>
+ val res = RexLiteralUtil.toFlinkInternalValue(constantKey.literal)
+ generateLiteral(ctx, res.f0, res.f1)
+ case fieldKey: FieldRef =>
+ generateInputAccess(
+ ctx,
+ inputType,
+ DEFAULT_INPUT1_TERM,
+ fieldKey.index,
+ nullableInput = false,
+ fieldCopy)
+ case _ =>
+ throw new CodeGenException("Invalid lookup key.")
+ }
+ }
+
+ /**
+ * Generates collector for join ([[Collector]])
+ *
+ * Differs from CommonCorrelate.generateCollector which has no real
condition because of
+ * FLINK-7865, here we should deal with outer join type when real conditions
filtered result.
+ */
+ def generateCollector(
+ ctx: CodeGeneratorContext,
+ inputRowType: RowType,
+ rightRowType: RowType,
+ resultRowType: RowType,
+ condition: Option[RexNode],
+ pojoFieldMapping: Option[Array[Int]],
+ retainHeader: Boolean = true):
GeneratedCollector[ListenableCollector[RowData]] = {
+
+ val inputTerm = DEFAULT_INPUT1_TERM
+ val rightInputTerm = DEFAULT_INPUT2_TERM
+
+ val exprGenerator = new ExprCodeGenerator(ctx, nullableInput = false)
+ .bindInput(rightRowType, inputTerm = rightInputTerm, inputFieldMapping =
pojoFieldMapping)
+
+ val rightResultExpr =
+ exprGenerator.generateConverterResultExpression(rightRowType,
classOf[GenericRowData])
+
+ val joinedRowTerm = CodeGenUtils.newName(ctx, "joinedRow")
+ ctx.addReusableOutputRecord(resultRowType, classOf[JoinedRowData],
joinedRowTerm)
+
+ val header = if (retainHeader) {
+ s"$joinedRowTerm.setRowKind($inputTerm.getRowKind());"
+ } else {
+ ""
+ }
+
+ val body =
+ s"""
+ |${rightResultExpr.code}
+ |$joinedRowTerm.replace($inputTerm, ${rightResultExpr.resultTerm});
+ |$header
+ |outputResult($joinedRowTerm);
+ """.stripMargin
+
+ val collectorCode = if (condition.isEmpty) {
+ body
+ } else {
+
+ val filterGenerator = new ExprCodeGenerator(ctx, nullableInput = false)
+ .bindInput(inputRowType, inputTerm)
+ .bindSecondInput(rightRowType, rightInputTerm, pojoFieldMapping)
+ val filterCondition = filterGenerator.generateExpression(condition.get)
+
+ s"""
+ |${filterCondition.code}
+ |if (${filterCondition.resultTerm}) {
+ | $body
+ |}
+ |""".stripMargin
+ }
+
+ generateTableFunctionCollectorForJoinTable(
+ ctx,
+ "JoinTableFuncCollector",
+ collectorCode,
+ inputRowType,
+ rightRowType,
+ inputTerm = inputTerm,
+ collectedTerm = rightInputTerm)
+ }
+
+ /**
+ * The only differences against
CollectorCodeGenerator.generateTableFunctionCollector is
+ * "super.collect" call is binding with collect join row in "body" code
+ */
+ private def generateTableFunctionCollectorForJoinTable(
Review Comment:
ditto
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala:
##########
@@ -112,108 +107,74 @@ object LookupJoinCodeGenerator {
lookupKeys: util.List[FunctionParam],
asyncLookupFunction: AsyncTableFunction[_],
functionName: String):
GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = {
-
- generateLookupFunction(
- classOf[AsyncFunction[RowData, AnyRef]],
+ FunctionCallCodeGenerator.generateAsyncFunctionCall(
tableConfig,
classLoader,
dataTypeFactory,
inputType,
tableSourceType,
returnType,
lookupKeys,
- classOf[AsyncTableFunction[_]],
asyncLookupFunction,
+ generateCallWithDataType(
+ dataTypeFactory,
+ functionName,
+ tableSourceType,
+ classOf[AsyncTableFunction[_]]),
functionName,
- fieldCopy = true, // always copy input field because of async buffer
- _.code
+ "AsyncLookupFunction"
)
}
- private def generateLookupFunction[F <: Function](
- generatedClass: Class[F],
- tableConfig: ReadableConfig,
- classLoader: ClassLoader,
+ private def generateCallWithDataType(
dataTypeFactory: DataTypeFactory,
- inputType: LogicalType,
- tableSourceType: LogicalType,
- returnType: LogicalType,
- lookupKeys: util.List[FunctionParam],
- lookupFunctionBase: Class[_],
- lookupFunction: UserDefinedFunction,
functionName: String,
- fieldCopy: Boolean,
- bodyCode: GeneratedExpression => String):
GeneratedTableFunctionWithDataType[F] = {
-
- val callContext =
- new LookupCallContext(dataTypeFactory, lookupFunction, inputType,
lookupKeys, tableSourceType)
-
- // create the final UDF for runtime
- val udf = UserDefinedFunctionHelper.createSpecializedFunction(
- functionName,
- lookupFunction,
- callContext,
- classOf[PlannerBase].getClassLoader,
- tableConfig,
- // no need to support expression evaluation at this point
- null)
-
- val inference =
- createLookupTypeInference(dataTypeFactory, callContext,
lookupFunctionBase, udf, functionName)
-
- val ctx = new CodeGeneratorContext(tableConfig, classLoader)
- val operands = prepareOperands(ctx, inputType, lookupKeys, fieldCopy)
-
- // TODO: filter all records when there are any nulls on the join key,
because
- // "IS NOT DISTINCT FROM" is not supported yet.
- // Note: AsyncPredictFunction or PredictFunction does not use Lookup
Syntax.
- val skipIfArgsNull = !lookupFunction.isInstanceOf[PredictFunction] &&
!lookupFunction
- .isInstanceOf[AsyncPredictFunction]
-
- val callWithDataType =
BridgingFunctionGenUtil.generateFunctionAwareCallWithDataType(
- ctx,
- operands,
- tableSourceType,
- inference,
- callContext,
- udf,
- functionName,
- skipIfArgsNull = skipIfArgsNull
- )
-
- val function = FunctionCodeGenerator.generateFunction(
- ctx,
- "LookupFunction",
- generatedClass,
- bodyCode(callWithDataType._1),
- returnType,
- inputType)
-
- GeneratedTableFunctionWithDataType(function, callWithDataType._2)
- }
-
- private def prepareOperands(
+ tableSourceType: LogicalType,
+ baseClass: Class[_]
+ ) = (
ctx: CodeGeneratorContext,
- inputType: LogicalType,
- lookupKeys: util.List[FunctionParam],
- fieldCopy: Boolean): Seq[GeneratedExpression] = {
-
- lookupKeys.asScala
- .map {
- case constantKey: Constant =>
- val res = RexLiteralUtil.toFlinkInternalValue(constantKey.literal)
- generateLiteral(ctx, res.f0, res.f1)
- case fieldKey: FieldRef =>
- generateInputAccess(
- ctx,
- inputType,
- DEFAULT_INPUT1_TERM,
- fieldKey.index,
- nullableInput = false,
- fieldCopy)
- case _ =>
- throw new CodeGenException("Invalid lookup key.")
- }
+ callContext: FunctionCallContext,
+ udf: UserDefinedFunction,
+ operands: Seq[GeneratedExpression]) => {
+ def inferCallWithDataType(
+ ctx: CodeGeneratorContext,
+ callContext: FunctionCallContext,
+ udf: UserDefinedFunction,
+ operands: Seq[GeneratedExpression],
+ legacy: Boolean,
+ e: Exception = null): (GeneratedExpression, DataType) = {
+ val inference = createLookupTypeInference(
+ dataTypeFactory,
+ callContext,
+ baseClass,
+ udf,
+ functionName,
+ legacy,
+ e)
+
+ // TODO: filter all records when there is any nulls on the join key,
because
+ // "IS NOT DISTINCT FROM" is not supported yet.
+ val callWithDataType =
BridgingFunctionGenUtil.generateFunctionAwareCallWithDataType(
+ ctx,
+ operands,
+ tableSourceType,
+ inference,
+ callContext,
+ udf,
+ functionName,
+ skipIfArgsNull = true
+ )
+ callWithDataType
+ }
+
+ try {
Review Comment:
Why do we move this out of `createLookupTypeInference`?
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCallCodeGenerator.scala:
##########
@@ -0,0 +1,344 @@
+/*
+ * 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
+
+import org.apache.flink.api.common.functions.{FlatMapFunction, Function,
OpenContext}
+import org.apache.flink.configuration.ReadableConfig
+import org.apache.flink.streaming.api.functions.async.AsyncFunction
+import org.apache.flink.table.catalog.DataTypeFactory
+import org.apache.flink.table.data.{GenericRowData, RowData}
+import org.apache.flink.table.data.utils.JoinedRowData
+import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction,
UserDefinedFunction, UserDefinedFunctionHelper}
+import
org.apache.flink.table.planner.codegen.CodeGenUtils.{boxedTypeTermForType,
className, newName, DEFAULT_COLLECTOR_TERM, DEFAULT_INPUT1_TERM,
DEFAULT_INPUT2_TERM}
+import
org.apache.flink.table.planner.codegen.GenerateUtils.{generateInputAccess,
generateLiteral}
+import org.apache.flink.table.planner.delegation.PlannerBase
+import org.apache.flink.table.planner.functions.inference.FunctionCallContext
+import org.apache.flink.table.planner.plan.utils.FunctionCallUtil.{Constant,
FieldRef, FunctionParam}
+import org.apache.flink.table.planner.plan.utils.RexLiteralUtil
+import org.apache.flink.table.runtime.collector.ListenableCollector
+import
org.apache.flink.table.runtime.collector.ListenableCollector.CollectListener
+import org.apache.flink.table.runtime.generated.{GeneratedCollector,
GeneratedFunction}
+import org.apache.flink.table.types.DataType
+import org.apache.flink.table.types.logical.{LogicalType, RowType}
+import org.apache.flink.util.Collector
+
+import org.apache.calcite.rex.RexNode
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+object FunctionCallCodeGenerator {
+
+ case class GeneratedTableFunctionWithDataType[F <: Function](
+ tableFunc: GeneratedFunction[F],
+ dataType: DataType)
+
+ /** Generates a sync function ([[TableFunction]]) call. */
+ def generateSyncFunctionCall(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ syncFunctionDefinition: TableFunction[_],
+ inferCall: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String,
+ fieldCopy: Boolean):
GeneratedTableFunctionWithDataType[FlatMapFunction[RowData, RowData]] = {
+
+ val bodyCode: GeneratedExpression => String = call => {
+ val resultCollectorTerm = call.resultTerm
+ s"""
+ |$resultCollectorTerm.setCollector($DEFAULT_COLLECTOR_TERM);
+ |${call.code}
+ |""".stripMargin
+ }
+
+ generateFunctionCall(
+ classOf[FlatMapFunction[RowData, RowData]],
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ functionOutputType,
+ collectorOutputType,
+ parameters,
+ syncFunctionDefinition,
+ inferCall,
+ functionName,
+ generateClassName,
+ fieldCopy,
+ bodyCode
+ )
+ }
+
+ /** Generates an async function ([[AsyncTableFunction]]) call. */
+ def generateAsyncFunctionCall(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ asyncFunctionDefinition: AsyncTableFunction[_],
+ generateCallWithDataType: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String
+ ): GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = {
+ generateFunctionCall(
+ classOf[AsyncFunction[RowData, AnyRef]],
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ functionOutputType,
+ collectorOutputType,
+ parameters,
+ asyncFunctionDefinition,
+ generateCallWithDataType,
+ functionName,
+ generateClassName,
+ fieldCopy = true,
+ _.code
+ )
+ }
+
+ private def generateFunctionCall[F <: Function](
+ generatedClass: Class[F],
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ functionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ functionDefinition: UserDefinedFunction,
+ generateCallWithDataType: (
+ CodeGeneratorContext,
+ FunctionCallContext,
+ UserDefinedFunction,
+ Seq[GeneratedExpression]) => (GeneratedExpression, DataType),
+ functionName: String,
+ generateClassName: String,
+ fieldCopy: Boolean,
+ bodyCode: GeneratedExpression => String):
GeneratedTableFunctionWithDataType[F] = {
+
+ val callContext =
+ new FunctionCallContext(
+ dataTypeFactory,
+ functionDefinition,
+ inputType,
+ parameters,
+ functionOutputType)
+
+ // create the final UDF for runtime
+ val udf = UserDefinedFunctionHelper.createSpecializedFunction(
+ functionName,
+ functionDefinition,
+ callContext,
+ classOf[PlannerBase].getClassLoader,
+ tableConfig,
+ // no need to support expression evaluation at this point
+ null
+ )
+
+ val ctx = new CodeGeneratorContext(tableConfig, classLoader)
+ val operands = prepareOperands(ctx, inputType, parameters, fieldCopy)
+
+ val callWithDataType: (GeneratedExpression, DataType) =
+ generateCallWithDataType(ctx, callContext, udf, operands)
+
+ val function = FunctionCodeGenerator.generateFunction(
+ ctx,
+ generateClassName,
+ generatedClass,
+ bodyCode(callWithDataType._1),
+ collectorOutputType,
+ inputType)
+
+ GeneratedTableFunctionWithDataType(function, callWithDataType._2)
+ }
+
+ private def prepareOperands(
+ ctx: CodeGeneratorContext,
+ inputType: LogicalType,
+ parameters: util.List[FunctionParam],
+ fieldCopy: Boolean): Seq[GeneratedExpression] = {
+
+ parameters.asScala
+ .map {
+ case constantKey: Constant =>
+ val res = RexLiteralUtil.toFlinkInternalValue(constantKey.literal)
+ generateLiteral(ctx, res.f0, res.f1)
+ case fieldKey: FieldRef =>
+ generateInputAccess(
+ ctx,
+ inputType,
+ DEFAULT_INPUT1_TERM,
+ fieldKey.index,
+ nullableInput = false,
+ fieldCopy)
+ case _ =>
+ throw new CodeGenException("Invalid lookup key.")
+ }
+ }
+
+ /**
+ * Generates collector for join ([[Collector]])
+ *
+ * Differs from CommonCorrelate.generateCollector which has no real
condition because of
+ * FLINK-7865, here we should deal with outer join type when real conditions
filtered result.
+ */
+ def generateCollector(
Review Comment:
Why this is not removed from `LookupJoinCodeGenerator`?
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMLPredictTableFunction.java:
##########
@@ -206,25 +200,15 @@ private Transformation<RowData> createModelPredict(
resultRowType,
mlPredictSpec.getFeatures(),
predictFunction,
- "MLPredict",
+
modelSpec.getContextResolvedModel().getIdentifier().asSummaryString(),
Review Comment:
Why do we change function name to model name?
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java:
##########
@@ -124,9 +119,6 @@ public Collector<RowData> getFetcherCollector() {
@Override
public void close() throws Exception {
- if (fetcher != null) {
Review Comment:
Missing calling super.close()?
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MLPredictCodeGenerator.scala:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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
+
+import org.apache.flink.api.common.functions.FlatMapFunction
+import org.apache.flink.configuration.ReadableConfig
+import org.apache.flink.streaming.api.functions.async.AsyncFunction
+import org.apache.flink.table.catalog.DataTypeFactory
+import org.apache.flink.table.data.RowData
+import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction,
UserDefinedFunction}
+import
org.apache.flink.table.planner.codegen.FunctionCallCodeGenerator.GeneratedTableFunctionWithDataType
+import org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil
+import org.apache.flink.table.planner.functions.inference.FunctionCallContext
+import org.apache.flink.table.planner.plan.utils.FunctionCallUtil.FunctionParam
+import org.apache.flink.table.runtime.collector.ListenableCollector
+import org.apache.flink.table.runtime.generated.{GeneratedCollector,
GeneratedFunction}
+import org.apache.flink.table.types.inference.{TypeInference, TypeStrategies,
TypeTransformations}
+import org.apache.flink.table.types.logical.{LogicalType, RowType}
+import org.apache.flink.table.types.utils.DataTypeUtils.transform
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+object MLPredictCodeGenerator {
+
+ /** Generates a predict function ([[TableFunction]]) */
+ def generateSyncPredictFunction(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ predictFunctionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ features: util.List[FunctionParam],
+ syncPredictFunction: TableFunction[_],
+ functionName: String,
+ fieldCopy: Boolean
+ ): GeneratedFunction[FlatMapFunction[RowData, RowData]] = {
+ FunctionCallCodeGenerator
+ .generateSyncFunctionCall(
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ predictFunctionOutputType,
+ collectorOutputType,
+ features,
+ syncPredictFunction,
+ generateCallWithDataType(functionName, predictFunctionOutputType),
+ functionName,
+ "PredictFunction",
+ fieldCopy
+ )
+ .tableFunc
+ }
+
+ /** Generates a async predict function ([[AsyncTableFunction]]) */
+ def generateAsyncPredictFunction(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ predictFunctionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ features: util.List[FunctionParam],
+ asyncPredictFunction: AsyncTableFunction[_],
+ functionName: String):
GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = {
+ FunctionCallCodeGenerator.generateAsyncFunctionCall(
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ predictFunctionOutputType,
+ collectorOutputType,
+ features,
+ asyncPredictFunction,
+ generateCallWithDataType(functionName, predictFunctionOutputType),
+ functionName,
+ "AsyncPredictFunction"
+ )
+ }
+
+ /** Generate a collector to collect to join the input row and predicted
results. */
+ def generateCollector(
+ ctx: CodeGeneratorContext,
+ inputRowType: RowType,
+ predictFunctionOutputType: RowType,
+ collectorOutputType: RowType
+ ): GeneratedCollector[ListenableCollector[RowData]] = {
+ FunctionCallCodeGenerator.generateCollector(
+ ctx,
+ inputRowType,
+ predictFunctionOutputType,
+ collectorOutputType,
+ Option.empty,
+ Option.empty
+ )
+ }
+
+ private def generateCallWithDataType(
+ functionName: String,
+ modelOutputType: LogicalType
+ ) = (
+ ctx: CodeGeneratorContext,
+ callContext: FunctionCallContext,
+ udf: UserDefinedFunction,
+ operands: Seq[GeneratedExpression]) => {
+ val inference = TypeInference
Review Comment:
why not call `udf.getTypeInference(dataTypeFactory)`?
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala:
##########
@@ -28,12 +28,14 @@ import org.apache.flink.table.data.utils.JoinedRowData
import org.apache.flink.table.functions.{AsyncLookupFunction,
AsyncPredictFunction, AsyncTableFunction, LookupFunction, PredictFunction,
TableFunction, UserDefinedFunction, UserDefinedFunctionHelper}
import org.apache.flink.table.planner.calcite.FlinkTypeFactory
import org.apache.flink.table.planner.codegen.CodeGenUtils._
+import
org.apache.flink.table.planner.codegen.FunctionCallCodeGenerator.GeneratedTableFunctionWithDataType
import org.apache.flink.table.planner.codegen.GenerateUtils._
import org.apache.flink.table.planner.codegen.Indenter.toISC
+import
org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator.generateCallWithDataType
Review Comment:
Seems import function from same class?
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/ml/AsyncMLPredictRunner.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.runtime.operators.ml;
+
+import org.apache.flink.api.common.functions.OpenContext;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.functions.async.CollectionSupplier;
+import org.apache.flink.streaming.api.functions.async.ResultFuture;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.JoinedRowData;
+import org.apache.flink.table.functions.AsyncPredictFunction;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.operators.AbstractAsyncFunctionRunner;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+
+/**
+ * Async function runner for {@link AsyncPredictFunction}, which takes the
generated function,
+ * instantiates it, and then calls its lifecycle methods.
+ */
+public class AsyncMLPredictRunner extends AbstractAsyncFunctionRunner<RowData>
{
+
+ private final int asyncBufferCapacity;
+
+ /**
+ * Buffers {@link ResultFuture} to avoid newInstance cost when processing
elements every time.
+ * We use {@link BlockingQueue} to make sure the head {@link
ResultFuture}s are available.
+ */
+ private transient BlockingQueue<JoinedRowResultFuture> resultFutureBuffer;
+
+ public AsyncMLPredictRunner(
+ GeneratedFunction<AsyncFunction<RowData, RowData>>
generatedFetcher,
+ int asyncBufferCapacity) {
+ super(generatedFetcher);
+ this.asyncBufferCapacity = asyncBufferCapacity;
+ }
+
+ @Override
+ public void open(OpenContext openContext) throws Exception {
+ super.open(openContext);
+ this.resultFutureBuffer = new ArrayBlockingQueue<>(asyncBufferCapacity
+ 1);
+ for (int i = 0; i < asyncBufferCapacity + 1; i++) {
+ JoinedRowResultFuture rf = new
JoinedRowResultFuture(resultFutureBuffer);
+ // add will throw exception immediately if the queue is full which
should never happen
+ resultFutureBuffer.add(rf);
+ }
+ registerMetric(getRuntimeContext().getMetricGroup());
+ }
+
+ @Override
+ public void asyncInvoke(RowData input, ResultFuture<RowData> resultFuture)
throws Exception {
+ try {
+ JoinedRowResultFuture buffer = resultFutureBuffer.take();
+ buffer.reset(input, resultFuture);
+ fetcher.asyncInvoke(input, buffer);
+ } catch (Throwable t) {
+ resultFuture.completeExceptionally(t);
+ }
+ }
+
+ private void registerMetric(MetricGroup metricGroup) {
+ metricGroup.gauge(
+ "ai_queue_length", () -> asyncBufferCapacity + 1 -
resultFutureBuffer.size());
Review Comment:
resultFutureBuffer size is fixed to be the same as asyncBufferCapacity?
--
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]