lihaosky commented on code in PR #27108:
URL: https://github.com/apache/flink/pull/27108#discussion_r2487958141
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java:
##########
@@ -682,6 +686,29 @@ public Table fromCall(Class<? extends UserDefinedFunction>
function, Object... a
operationTreeBuilder.tableFunction(Expressions.call(function,
arguments)));
}
+ @Override
+ public Model fromModelPath(String modelPath) {
+ UnresolvedIdentifier unresolvedIdentifier =
getParser().parseIdentifier(modelPath);
+ ObjectIdentifier modelIdentifier =
catalogManager.qualifyIdentifier(unresolvedIdentifier);
+ return catalogManager
+ .getModel(modelIdentifier)
+ .map(this::createModel)
+ .orElseThrow(
+ () ->
+ new ValidationException(
Review Comment:
I'm following same pattern of `from()` for table
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ModelReferenceExpression.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.expressions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.Model;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.catalog.ContextResolvedModel;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * A reference to a {@link Model} in an expression context.
+ *
+ * <p>This expression is used when a model needs to be passed as an argument
to functions or
+ * operations that accept model references. It wraps a model object and
provides the necessary
+ * expression interface for use in the Table API expression system.
+ *
+ * <p>The expression carries a string representation of the model and uses a
special data type to
+ * indicate that this is a model reference rather than a regular data value.
+ */
+@Internal
+public final class ModelReferenceExpression implements ResolvedExpression {
+
+ private final String name;
+ private final ContextResolvedModel model;
+ private final TableEnvironment env;
Review Comment:
It's used in
https://github.com/apache/flink/pull/27108/files#diff-7c9c46d35bb24b9f87d4c993274ef8c0054894c86c72c8afcc0c4f6398562fdeR1570.
Same pattern for `TableReferenceExpression`
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java:
##########
@@ -682,6 +686,29 @@ public Table fromCall(Class<? extends UserDefinedFunction>
function, Object... a
operationTreeBuilder.tableFunction(Expressions.call(function,
arguments)));
}
+ @Override
+ public Model fromModelPath(String modelPath) {
+ UnresolvedIdentifier unresolvedIdentifier =
getParser().parseIdentifier(modelPath);
+ ObjectIdentifier modelIdentifier =
catalogManager.qualifyIdentifier(unresolvedIdentifier);
+ return catalogManager
+ .getModel(modelIdentifier)
+ .map(this::createModel)
+ .orElseThrow(
+ () ->
+ new ValidationException(
+ String.format(
+ "Model %s was not found.",
unresolvedIdentifier)));
+ }
+
+ @Override
+ public Model from(ModelDescriptor descriptor) {
+ Preconditions.checkNotNull(descriptor, "Model descriptor must not be
null.");
Review Comment:
I'm following same pattern for table
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ModelImpl.java:
##########
@@ -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 Expression.lit() to create a MAP
literal since
+ // lit() is not serializable to sql.
+ if (options.isEmpty()) {
+ return tableEnvironment.fromCall(
+ "ML_PREDICT",
+ 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(
+ "ML_PREDICT",
+ table.asArgument("INPUT"),
+ this.asArgument("MODEL"),
+ Expressions.descriptor(inputColumns).asArgument("ARGS"),
+ Expressions.map(
Review Comment:
I can try it if it also works.
--
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]