lihaosky commented on code in PR #27108: URL: https://github.com/apache/flink/pull/27108#discussion_r2491424397
########## 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 mainly for validation I think. See https://github.com/apache/flink/blob/master/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java#L1527-L1531 https://github.com/apache/flink/blob/master/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/TableReferenceExpression.java#L46-L48 I think for `FieldReferenceExpression`, it just contains field name and type, no need to validate environment I guess. How about I add more comments here similar to table? ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java: ########## @@ -235,27 +255,96 @@ public RexNode visit(TypeLiteralExpression typeLiteral) { } @Override - public RexNode visit(Expression other) { + public RexNode visit(TableReferenceExpression tableRef) { + final LogicalType tableArgType = tableRef.getOutputDataType().getLogicalType(); + final RelDataType rowType = typeFactory.buildRelNodeRowType((RowType) tableArgType); + final int[] partitionKeys; + if (tableRef.getQueryOperation() instanceof PartitionQueryOperation) { + final PartitionQueryOperation partitionOperation = + (PartitionQueryOperation) tableRef.getQueryOperation(); + partitionKeys = partitionOperation.getPartitionKeys(); + } else { + partitionKeys = new int[0]; + } + final RexTableArgCall tableArgCall = + new RexTableArgCall(rowType, inputStack.size(), partitionKeys, new int[0]); + inputStack.add(relBuilder.build()); Review Comment: It's hard to do since `inputStack` depends on this call. Maybe that's why `TableReferenceExpression` was handled in `QueryOperationConverter` in the first place ########## flink-python/pyflink/table/tests/test_table_environment_completeness.py: ########## @@ -44,6 +44,7 @@ def excluded_methods(cls): "from", "registerFunction", "fromCall", + "fromModelPath", Review Comment: Done: https://issues.apache.org/jira/browse/FLINK-38623 ########## flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ModelReferenceExpression.java: ########## @@ -0,0 +1,153 @@ +/* + * 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.api.ValidationException; +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; + + public ModelReferenceExpression(String name, ContextResolvedModel model, TableEnvironment env) { + this.name = Preconditions.checkNotNull(name); + this.model = Preconditions.checkNotNull(model); + this.env = Preconditions.checkNotNull(env); + } + + /** + * Returns the name of this model reference. + * + * @return the model reference name + */ + public String getName() { + return name; + } + + /** + * Returns the ContextResolvedModel associated with this model reference. + * + * @return the query context resolved model + */ + public ContextResolvedModel getModel() { + return model; + } + + public @Nullable TableEnvironment getTableEnvironment() { + return env; + } + + /** + * Returns the input data type expected by this model reference. + * + * <p>This method extracts the input data type from the model's input schema, which describes + * the structure and data types that the model expects for inference operations. + * + * @return the input data type expected by the model + */ + public DataType getInputDataType() { + return DataTypeUtils.fromResolvedSchemaPreservingTimeAttributes( + model.getResolvedModel().getResolvedInputSchema()); + } + + @Override + public DataType getOutputDataType() { + return DataTypeUtils.fromResolvedSchemaPreservingTimeAttributes( + model.getResolvedModel().getResolvedOutputSchema()); + } + + @Override + public List<ResolvedExpression> getResolvedChildren() { + return Collections.emptyList(); + } + + @Override + public String asSerializableString(SqlFactory sqlFactory) { + if (model.isAnonymous()) { + throw new ValidationException("Anonymous models cannot be serialized."); + } + + return "MODEL " + model.getIdentifier().asSerializableString(); Review Comment: The name are not used for `TableReferenceExpression` and literal argument name as well. These are not serialized as named argument call. I don't see this is restored from serialized string, looks mainly to convert it sql query looks -- 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]
