-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-38435][table] Refactor codegen and runner for MLPredict and Lo… #27041
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,8 +31,6 @@ | |
| import org.apache.flink.table.api.TableException; | ||
| import org.apache.flink.table.catalog.DataTypeFactory; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.data.conversion.DataStructureConverter; | ||
| import org.apache.flink.table.data.conversion.DataStructureConverters; | ||
| import org.apache.flink.table.functions.AsyncPredictFunction; | ||
| import org.apache.flink.table.functions.PredictFunction; | ||
| import org.apache.flink.table.functions.UserDefinedFunction; | ||
|
|
@@ -41,8 +39,8 @@ | |
| import org.apache.flink.table.ml.PredictRuntimeProvider; | ||
| import org.apache.flink.table.planner.calcite.FlinkContext; | ||
| import org.apache.flink.table.planner.codegen.CodeGeneratorContext; | ||
| import org.apache.flink.table.planner.codegen.FilterCodeGenerator; | ||
| import org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator; | ||
| import org.apache.flink.table.planner.codegen.FunctionCallCodeGenerator; | ||
| import org.apache.flink.table.planner.codegen.MLPredictCodeGenerator; | ||
| import org.apache.flink.table.planner.delegation.PlannerBase; | ||
| import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; | ||
| import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; | ||
|
|
@@ -55,18 +53,15 @@ | |
| import org.apache.flink.table.planner.plan.nodes.exec.spec.ModelSpec; | ||
| import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; | ||
| import org.apache.flink.table.planner.plan.utils.FunctionCallUtil; | ||
| import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; | ||
| import org.apache.flink.table.runtime.collector.ListenableCollector; | ||
| import org.apache.flink.table.runtime.collector.TableFunctionResultFuture; | ||
| import org.apache.flink.table.runtime.functions.ml.ModelPredictRuntimeProviderContext; | ||
| import org.apache.flink.table.runtime.generated.GeneratedCollector; | ||
| import org.apache.flink.table.runtime.generated.GeneratedFunction; | ||
| import org.apache.flink.table.runtime.generated.GeneratedResultFuture; | ||
| import org.apache.flink.table.runtime.operators.join.lookup.AsyncLookupJoinRunner; | ||
| import org.apache.flink.table.runtime.operators.join.lookup.LookupJoinRunner; | ||
| import org.apache.flink.table.runtime.typeutils.InternalSerializers; | ||
| import org.apache.flink.table.runtime.operators.ml.AsyncMLPredictRunner; | ||
| import org.apache.flink.table.runtime.operators.ml.MLPredictRunner; | ||
| import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; | ||
| import org.apache.flink.table.types.logical.RowType; | ||
| import org.apache.flink.util.Preconditions; | ||
|
|
||
| import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; | ||
| import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; | ||
|
|
@@ -75,7 +70,6 @@ | |
|
|
||
| import java.util.Collections; | ||
| import java.util.List; | ||
| import java.util.Optional; | ||
|
|
||
| /** Stream {@link ExecNode} for {@code ML_PREDICT}. */ | ||
| @ExecNodeMetadata( | ||
|
|
@@ -197,7 +191,7 @@ private Transformation<RowData> createModelPredict( | |
| RowType resultRowType, | ||
| PredictFunction predictFunction) { | ||
| GeneratedFunction<FlatMapFunction<RowData, RowData>> generatedFetcher = | ||
| LookupJoinCodeGenerator.generateSyncLookupFunction( | ||
| MLPredictCodeGenerator.generateSyncPredictFunction( | ||
| config, | ||
| classLoader, | ||
| dataTypeFactory, | ||
|
|
@@ -206,25 +200,15 @@ private Transformation<RowData> createModelPredict( | |
| resultRowType, | ||
| mlPredictSpec.getFeatures(), | ||
| predictFunction, | ||
| "MLPredict", | ||
| modelSpec.getContextResolvedModel().getIdentifier().asSummaryString(), | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we change function name to model name?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The field here is used by UserDefinedFunctionHelper#createSpecializedFunction. When the model doesn't implement required interface, an exception should be thrown to indicate which function is illegal. Therefore, the origin implementation is not correct.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sorry didn't get it. Which required interface? Is it to improve error message?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
| config.get(PipelineOptions.OBJECT_REUSE)); | ||
| GeneratedCollector<ListenableCollector<RowData>> generatedCollector = | ||
| LookupJoinCodeGenerator.generateCollector( | ||
| MLPredictCodeGenerator.generateCollector( | ||
| new CodeGeneratorContext(config, classLoader), | ||
| inputRowType, | ||
| modelOutputType, | ||
| (RowType) getOutputType(), | ||
| JavaScalaConversionUtil.toScala(Optional.empty()), | ||
| JavaScalaConversionUtil.toScala(Optional.empty()), | ||
| true); | ||
| LookupJoinRunner mlPredictRunner = | ||
| new LookupJoinRunner( | ||
| generatedFetcher, | ||
| generatedCollector, | ||
| FilterCodeGenerator.generateFilterCondition( | ||
| config, classLoader, null, inputRowType), | ||
| false, | ||
| modelOutputType.getFieldCount()); | ||
| (RowType) getOutputType()); | ||
| MLPredictRunner mlPredictRunner = new MLPredictRunner(generatedFetcher, generatedCollector); | ||
| SimpleOperatorFactory<RowData> operatorFactory = | ||
| SimpleOperatorFactory.of(new ProcessOperator<>(mlPredictRunner)); | ||
| return ExecNodeUtil.createOneInputTransformation( | ||
|
|
@@ -246,9 +230,9 @@ private Transformation<RowData> createAsyncModelPredict( | |
| RowType modelOutputType, | ||
| RowType resultRowType, | ||
| AsyncPredictFunction asyncPredictFunction) { | ||
| LookupJoinCodeGenerator.GeneratedTableFunctionWithDataType<AsyncFunction<RowData, Object>> | ||
| FunctionCallCodeGenerator.GeneratedTableFunctionWithDataType<AsyncFunction<RowData, Object>> | ||
| generatedFuncWithType = | ||
| LookupJoinCodeGenerator.generateAsyncLookupFunction( | ||
| MLPredictCodeGenerator.generateAsyncPredictFunction( | ||
| config, | ||
| classLoader, | ||
| dataTypeFactory, | ||
|
|
@@ -257,29 +241,14 @@ private Transformation<RowData> createAsyncModelPredict( | |
| resultRowType, | ||
| mlPredictSpec.getFeatures(), | ||
| asyncPredictFunction, | ||
| "AsyncMLPredict"); | ||
|
|
||
| GeneratedResultFuture<TableFunctionResultFuture<RowData>> generatedResultFuture = | ||
| LookupJoinCodeGenerator.generateTableAsyncCollector( | ||
| config, | ||
| classLoader, | ||
| "TableFunctionResultFuture", | ||
| inputRowType, | ||
| modelOutputType, | ||
| JavaScalaConversionUtil.toScala(Optional.empty())); | ||
|
|
||
| DataStructureConverter<?, ?> fetcherConverter = | ||
| DataStructureConverters.getConverter(generatedFuncWithType.dataType()); | ||
| modelSpec | ||
| .getContextResolvedModel() | ||
| .getIdentifier() | ||
| .asSummaryString()); | ||
| AsyncFunction<RowData, RowData> asyncFunc = | ||
| new AsyncLookupJoinRunner( | ||
| generatedFuncWithType.tableFunc(), | ||
| (DataStructureConverter<RowData, Object>) fetcherConverter, | ||
| generatedResultFuture, | ||
| FilterCodeGenerator.generateFilterCondition( | ||
| config, classLoader, null, inputRowType), | ||
| InternalSerializers.create(modelOutputType), | ||
| false, | ||
| asyncOptions.asyncBufferCapacity); | ||
| new AsyncMLPredictRunner( | ||
| (GeneratedFunction) generatedFuncWithType.tableFunc(), | ||
| Preconditions.checkNotNull(asyncOptions).asyncBufferCapacity); | ||
| return ExecNodeUtil.createOneInputTransformation( | ||
| inputTransformation, | ||
| createTransformationMeta(ML_PREDICT_TRANSFORMATION, config), | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.