-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-38104][table] add table api support for model ml_predict #27108
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
Open
lihaosky
wants to merge
7
commits into
apache:master
Choose a base branch
from
confluentinc:model-table-api
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+1,372
−88
Open
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
2973b01
[FLINK-38104][table] add table api support for model ml_predict
lihaosky ca0700b
fix
lihaosky bfab15b
fix
lihaosky b7d9a51
fix
lihaosky abf8b08
fix
lihaosky 208c58d
comments
lihaosky a21c25b
comments
lihaosky File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -44,6 +44,7 @@ def excluded_methods(cls): | |
| "from", | ||
| "registerFunction", | ||
| "fromCall", | ||
| "fromModelPath", | ||
| } | ||
|
|
||
|
|
||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
146 changes: 146 additions & 0 deletions
146
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Model.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.api; | ||
|
|
||
| import org.apache.flink.annotation.PublicEvolving; | ||
| import org.apache.flink.table.api.config.MLPredictRuntimeConfigOptions; | ||
| import org.apache.flink.table.catalog.ResolvedSchema; | ||
| import org.apache.flink.types.ColumnList; | ||
|
|
||
| import java.util.Map; | ||
|
|
||
| /** | ||
| * The {@link Model} object is the core abstraction for ML model resources in the Table API. | ||
| * | ||
| * <p>A {@link Model} object describes a machine learning model resource that can be used for | ||
| * inference operations. It provides methods to perform prediction on data tables. | ||
| * | ||
| * <p>The {@link Model} interface offers main operations: | ||
| * | ||
| * <ul> | ||
| * <li>{@link #predict(Table, ColumnList)} - Applies the model to make predictions on input data | ||
| * </ul> | ||
| * | ||
| * <p>{@code ml_predict} operation supports runtime options for configuring execution parameters | ||
| * such as asynchronous execution mode. | ||
| * | ||
| * <p>Every {@link Model} object has input and output schemas that describe the expected data | ||
| * structure for model operations, available through {@link #getResolvedInputSchema()} and {@link | ||
| * #getResolvedOutputSchema()}. | ||
| * | ||
| * <p>Example usage: | ||
| * | ||
| * <pre>{@code | ||
| * Model model = tableEnv.fromModelPath("my_model"); | ||
| * | ||
| * // Simple prediction | ||
| * Table predictions = model.predict(inputTable, ColumnList.of("feature1", "feature2")); | ||
| * | ||
| * // Prediction with options | ||
| * Map<String, String> options = Map.of("max-concurrent-operations", "100", "timeout", "30s", "async", "true"); | ||
| * Table predictions = model.predict(inputTable, ColumnList.of("feature1", "feature2"), options); | ||
| * }</pre> | ||
| */ | ||
| @PublicEvolving | ||
| public interface Model { | ||
|
|
||
| /** | ||
| * Returns the resolved input schema of this model. | ||
| * | ||
| * <p>The input schema describes the structure and data types of the input columns that the | ||
| * model expects for inference operations. | ||
| * | ||
| * @return the resolved input schema. | ||
| */ | ||
| ResolvedSchema getResolvedInputSchema(); | ||
|
|
||
| /** | ||
| * Returns the resolved output schema of this model. | ||
| * | ||
| * <p>The output schema describes the structure and data types of the output columns that the | ||
| * model produces during inference operations. | ||
| * | ||
| * @return the resolved output schema. | ||
| */ | ||
| ResolvedSchema getResolvedOutputSchema(); | ||
|
|
||
| /** | ||
| * Performs prediction on the given table using specified input columns. | ||
| * | ||
| * <p>This method applies the model to the input data to generate predictions. The input columns | ||
| * must match the model's expected input schema. | ||
| * | ||
| * <p>Example: | ||
| * | ||
| * <pre>{@code | ||
| * Table predictions = model.predict(inputTable, ColumnList.of("feature1", "feature2")); | ||
| * }</pre> | ||
| * | ||
| * @param table the input table containing data for prediction | ||
| * @param inputColumns the columns from the input table to use as model input | ||
| * @return a table containing the input data along with prediction results | ||
| */ | ||
| Table predict(Table table, ColumnList inputColumns); | ||
|
|
||
| /** | ||
| * Performs prediction on the given table using specified input columns with runtime options. | ||
| * | ||
| * <p>This method applies the model to the input data to generate predictions with additional | ||
| * runtime configuration options such as max-concurrent-operations, timeout, and execution mode | ||
| * settings. | ||
| * | ||
| * <p>For Common runtime options, see {@link MLPredictRuntimeConfigOptions}. | ||
| * | ||
| * <p>Example: | ||
| * | ||
| * <pre>{@code | ||
| * Map<String, String> options = Map.of("max-concurrent-operations", "100", "timeout", "30s", "async", "true"); | ||
| * Table predictions = model.predict(inputTable, | ||
| * ColumnList.of("feature1", "feature2"), options); | ||
| * }</pre> | ||
| * | ||
| * @param table the input table containing data for prediction | ||
| * @param inputColumns the columns from the input table to use as model input | ||
| * @param options runtime options for configuring the prediction operation | ||
| * @return a table containing the input data along with prediction results | ||
| */ | ||
| Table predict(Table table, ColumnList inputColumns, Map<String, String> options); | ||
|
|
||
| /** | ||
| * Converts this model object into a named argument. | ||
| * | ||
| * <p>This method is intended for use in function calls that accept model arguments, | ||
| * particularly in process table functions (PTFs) or other operations that work with models. | ||
| * | ||
| * <p>Example: | ||
| * | ||
| * <pre>{@code | ||
| * env.fromCall( | ||
| * "ML_PREDICT", | ||
| * inputTable.asArgument("INPUT"), | ||
| * model.asArgument("MODEL"), | ||
| * Expressions.descriptor(ColumnList.of("feature1", "feature2")).asArgument("ARGS") | ||
| * ) | ||
| * }</pre> | ||
| * | ||
| * @param name the name to assign to this model argument | ||
| * @return an expression that can be passed to functions expecting model arguments | ||
| */ | ||
| ApiExpression asArgument(String name); | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
119 changes: 119 additions & 0 deletions
119
...ble/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ModelImpl.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,119 @@ | ||
| /* | ||
| * 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.api.internal; | ||
|
|
||
| import org.apache.flink.annotation.Internal; | ||
| import org.apache.flink.table.api.ApiExpression; | ||
| import org.apache.flink.table.api.Expressions; | ||
| import org.apache.flink.table.api.Model; | ||
| import org.apache.flink.table.api.Table; | ||
| import org.apache.flink.table.api.TableEnvironment; | ||
| import org.apache.flink.table.catalog.ContextResolvedModel; | ||
| import org.apache.flink.table.catalog.ResolvedSchema; | ||
| import org.apache.flink.table.expressions.ApiExpressionUtils; | ||
| import org.apache.flink.table.functions.BuiltInFunctionDefinitions; | ||
| import org.apache.flink.types.ColumnList; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.Map; | ||
|
|
||
| import static org.apache.flink.table.api.Expressions.lit; | ||
|
|
||
| /** Implementation of {@link Model} that works with the Table API. */ | ||
| @Internal | ||
| public class ModelImpl implements Model { | ||
|
|
||
| private final TableEnvironmentInternal tableEnvironment; | ||
| private final ContextResolvedModel model; | ||
|
|
||
| private ModelImpl(TableEnvironmentInternal tableEnvironment, ContextResolvedModel model) { | ||
| this.tableEnvironment = tableEnvironment; | ||
| this.model = model; | ||
| } | ||
|
|
||
| public static ModelImpl createModel( | ||
| TableEnvironmentInternal tableEnvironment, ContextResolvedModel model) { | ||
| return new ModelImpl(tableEnvironment, model); | ||
| } | ||
|
|
||
| public ContextResolvedModel getModel() { | ||
| return model; | ||
| } | ||
|
|
||
| @Override | ||
| public ResolvedSchema getResolvedInputSchema() { | ||
| return model.getResolvedModel().getResolvedInputSchema(); | ||
| } | ||
|
|
||
| @Override | ||
| public ResolvedSchema getResolvedOutputSchema() { | ||
| return model.getResolvedModel().getResolvedOutputSchema(); | ||
| } | ||
|
|
||
| public TableEnvironment getTableEnv() { | ||
| return tableEnvironment; | ||
| } | ||
|
|
||
| @Override | ||
| public Table predict(Table table, ColumnList inputColumns) { | ||
| return predict(table, inputColumns, Map.of()); | ||
| } | ||
|
|
||
| @Override | ||
| public Table predict(Table table, ColumnList inputColumns, Map<String, String> options) { | ||
| // Use Expressions.map() instead of Expressions.lit() to create a MAP literal since | ||
| // lit() is not serializable to sql. | ||
| if (options.isEmpty()) { | ||
| return tableEnvironment.fromCall( | ||
| BuiltInFunctionDefinitions.ML_PREDICT.getName(), | ||
| table.asArgument("INPUT"), | ||
| this.asArgument("MODEL"), | ||
| Expressions.descriptor(inputColumns).asArgument("ARGS")); | ||
| } | ||
| ArrayList<String> configKVs = new ArrayList<>(); | ||
| options.forEach( | ||
| (k, v) -> { | ||
| configKVs.add(k); | ||
| configKVs.add(v); | ||
| }); | ||
| return tableEnvironment.fromCall( | ||
| BuiltInFunctionDefinitions.ML_PREDICT.getName(), | ||
| table.asArgument("INPUT"), | ||
| this.asArgument("MODEL"), | ||
| Expressions.descriptor(inputColumns).asArgument("ARGS"), | ||
| Expressions.map( | ||
lihaosky marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| configKVs.get(0), | ||
| configKVs.get(1), | ||
| configKVs.subList(2, configKVs.size()).toArray()) | ||
| .asArgument("CONFIG")); | ||
| } | ||
|
|
||
| @Override | ||
| public ApiExpression asArgument(String name) { | ||
| return new ApiExpression( | ||
| ApiExpressionUtils.unresolvedCall( | ||
| BuiltInFunctionDefinitions.ASSIGNMENT, | ||
| lit(name), | ||
| ApiExpressionUtils.modelRef(name, this))); | ||
| } | ||
|
|
||
| public TableEnvironment getTableEnvironment() { | ||
| return tableEnvironment; | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Create a tickect about this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done: https://issues.apache.org/jira/browse/FLINK-38623