lihaosky commented on code in PR #25834: URL: https://github.com/apache/flink/pull/25834#discussion_r1964269034
########## flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterModelReset.java: ########## @@ -0,0 +1,76 @@ +/* + * 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.sql.parser.ddl; + +import org.apache.flink.sql.parser.SqlUnparseUtils; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlLiteral; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.util.ImmutableNullableList; +import org.apache.calcite.util.NlsString; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.Objects.requireNonNull; + +/** + * ALTER MODEL [IF EXISTS] [[catalogName.] dataBasesName.]modelName RESET ( 'key1' [, 'key2']...). + */ +public class SqlAlterModelReset extends SqlAlterModel { + private final SqlNodeList optionKeyList; + + public SqlAlterModelReset( + SqlParserPos pos, + SqlIdentifier modelName, + boolean ifModelExists, + SqlNodeList optionKeyList) { + super(pos, modelName, ifModelExists); + this.optionKeyList = requireNonNull(optionKeyList, "optionKeyList should not be null"); + } + + @Override + public List<SqlNode> getOperandList() { + return ImmutableNullableList.of(modelName, optionKeyList); Review Comment: I think it's copied from https://github.com/apache/flink/blob/master/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterTableReset.java#L54. Maybe to avoid using `ImmutableList.of` from guava? I can change it to `ImmutableList.of` ########## flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/DescribeModelOperation.java: ########## @@ -0,0 +1,156 @@ +/* + * 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.operations; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ContextResolvedModel; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; + +import org.apache.flink.shaded.guava32.com.google.common.collect.Streams; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.table.api.internal.TableResultUtils.buildTableResult; + +/** + * Operation to describe a DESCRIBE MODEL [EXTENDED] [[catalogName.] dataBasesName.]sqlIdentifier + * statement. + */ +@Internal +public class DescribeModelOperation implements Operation, ExecutableOperation { + + private final ObjectIdentifier sqlIdentifier; + private final boolean isExtended; + + public DescribeModelOperation(ObjectIdentifier sqlIdentifier, boolean isExtended) { + this.sqlIdentifier = sqlIdentifier; + this.isExtended = isExtended; + } + + public ObjectIdentifier getSqlIdentifier() { + return sqlIdentifier; + } + + public boolean isExtended() { + return isExtended; + } + + @Override + public String asSummaryString() { + Map<String, Object> params = new LinkedHashMap<>(); + params.put("identifier", sqlIdentifier); + params.put("isExtended", isExtended); + return OperationUtils.formatWithChildren( + "DESCRIBE MODEL", params, Collections.emptyList(), Operation::asSummaryString); + } + + @Override + public TableResultInternal execute(Context ctx) { + // DESCRIBE MODEL <model> shows input/output schema if any. + Optional<ContextResolvedModel> result = ctx.getCatalogManager().getModel(sqlIdentifier); + if (result.isEmpty()) { + throw new ValidationException( + String.format( + "Model with the identifier '%s' doesn't exist.", + sqlIdentifier.asSummaryString())); + } + + ResolvedSchema inputSchema = result.get().getResolvedModel().getResolvedInputSchema(); + ResolvedSchema outputSchema = result.get().getResolvedModel().getResolvedOutputSchema(); + Object[][] rows = generateModelColumnsRows(inputSchema, outputSchema); + + boolean nonComments = isSchemaNonColumnComments(inputSchema, outputSchema); + return buildTableResult( + generateTableColumnsNames(nonComments), + generateTableColumnsDataTypes(nonComments), + rows); + } + + private Object[][] generateModelColumnsRows( + ResolvedSchema inputSchema, ResolvedSchema outputSchema) { + boolean nonComments = isSchemaNonColumnComments(inputSchema, outputSchema); + + return Streams.concat( + inputSchema.getColumns().stream() + .map((c) -> buildSingleRow(c, nonComments, true)), + outputSchema.getColumns().stream() + .map((c) -> buildSingleRow(c, nonComments, false))) + .toArray(Object[][]::new); + } + + private Object[] buildSingleRow(Column c, boolean nonComments, boolean isInput) { + final LogicalType logicalType = c.getDataType().getLogicalType(); + final ArrayList<Object> result = + new ArrayList<>( + Arrays.asList( + c.getName(), + logicalType.copy(true).asSummaryString(), Review Comment: Next column is specifically to indicate whether it can be nullable. So we don't display `not null` here ########## flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java: ########## @@ -1512,18 +1513,52 @@ public void createTemporaryModel( /** * Alters a model in a given fully qualified path. * - * @param modelChange The model containing only changes + * @param newModel The new model containing changes. It could be null if the model to alter + * doesn't exist + * @param modelChanges The changes to apply to the model. * @param objectIdentifier The fully qualified path where to alter the model. * @param ignoreIfNotExists If false exception will be thrown if the model to be altered does * not exist. */ public void alterModel( - CatalogModel modelChange, + @Nullable CatalogModel newModel, + List<ModelChange> modelChanges, ObjectIdentifier objectIdentifier, boolean ignoreIfNotExists) { execute( (catalog, path) -> { - ResolvedCatalogModel resolvedModel = resolveCatalogModel(modelChange); + ResolvedCatalogModel resolvedModel = + newModel == null ? null : resolveCatalogModel(newModel); + catalog.alterModel(path, resolvedModel, modelChanges, ignoreIfNotExists); Review Comment: In `altertable` call, if the table doesn't exist, the operation is a `NopOperation`: https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java#L409. This leads to problem where we don't know the original sql operation and properly give error message etc. So in `alterModel` it's still `altermodelOperation` and null indicates original model doesn't exist. It can be handled by downstream ########## flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterModelRename.java: ########## @@ -0,0 +1,67 @@ +/* + * 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.sql.parser.ddl; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.util.ImmutableNullableList; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +/** ALTER MODEL [IF EXISTS] [[catalogName.] dataBasesName.]modelName RENAME TO newModelName. */ +public class SqlAlterModelRename extends SqlAlterModel { + + private final SqlIdentifier newModelName; + + public SqlAlterModelRename( + SqlParserPos pos, + SqlIdentifier modelName, + SqlIdentifier newModelName, + boolean ifModelExists) { + super(pos, modelName, ifModelExists); + this.newModelName = requireNonNull(newModelName, "newModelName should not be null"); + } + + public SqlIdentifier getNewModelName() { + return newModelName; + } + + public String[] fullNewModelName() { + if (newModelName != null) { + return newModelName.names.toArray(new String[0]); + } + return new String[0]; Review Comment: Nope. Let me remove this ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterModelSetConverter.java: ########## @@ -0,0 +1,105 @@ +/* + * 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.planner.operations.converters; + +import org.apache.flink.sql.parser.ddl.SqlAlterModelSet; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.ContextResolvedModel; +import org.apache.flink.table.catalog.ModelChange; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogModel; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.ddl.AlterModelChangeOperation; +import org.apache.flink.table.planner.utils.OperationConverterUtils; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.stream.Collectors; + +/** A converter for {@link org.apache.flink.sql.parser.ddl.SqlAlterModelSet}. */ +public class SqlAlterModelSetConverter implements SqlNodeConverter<SqlAlterModelSet> { + + @Override + public Operation convertSqlNode(SqlAlterModelSet sqlAlterModelSet, ConvertContext context) { + final CatalogManager catalogManager = context.getCatalogManager(); + UnresolvedIdentifier unresolvedIdentifier = + UnresolvedIdentifier.of(sqlAlterModelSet.fullModelName()); + ObjectIdentifier modelIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + Optional<ContextResolvedModel> optionalCatalogModel = + catalogManager.getModel(modelIdentifier); + if (optionalCatalogModel.isEmpty() || optionalCatalogModel.get().isTemporary()) { + if (optionalCatalogModel.isEmpty()) { + if (!sqlAlterModelSet.ifModelExists()) { + throw new ValidationException( + String.format("Model %s doesn't exist.", modelIdentifier)); + } + } else if (optionalCatalogModel.get().isTemporary()) { + throw new ValidationException( + String.format("Model %s is a temporary model.", modelIdentifier)); + } + } + ResolvedCatalogModel existingModel = + optionalCatalogModel.map(ContextResolvedModel::getResolvedModel).orElse(null); Review Comment: I can take a look at this -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org