lihaosky commented on code in PR #27108:
URL: https://github.com/apache/flink/pull/27108#discussion_r2504542394


##########
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 Expressions.lit() to create a MAP 
literal since
+        // lit() is not serializable to sql.
+        if (options.isEmpty()) {
+            return tableEnvironment.fromCall(
+                    BuiltInFunctionDefinitions.ML_PREDICT.getName(),

Review Comment:
   I can get rid of `getName`. `TableEnvironment.fromCall` has additional 
checks like
   ```
   tableReferenceChecker.check(arguments);
   ```
   which seems useful though



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

Reply via email to