LadyForest commented on code in PR #20652: URL: https://github.com/apache/flink/pull/20652#discussion_r1045810306
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/AlterTableSchemaUtil.java: ########## @@ -0,0 +1,525 @@ +/* + * 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; + +import org.apache.flink.sql.parser.ddl.SqlAlterTableAdd; +import org.apache.flink.sql.parser.ddl.SqlAlterTableModify; +import org.apache.flink.sql.parser.ddl.SqlAlterTableSchema; +import org.apache.flink.sql.parser.ddl.SqlTableColumn; +import org.apache.flink.sql.parser.ddl.SqlWatermark; +import org.apache.flink.sql.parser.ddl.constraint.SqlTableConstraint; +import org.apache.flink.sql.parser.ddl.position.SqlTableColumnPosition; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.SchemaResolver; +import org.apache.flink.table.expressions.SqlCallExpression; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.types.AbstractDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.validate.SqlValidator; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.toLogicalType; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** Util to convert {@link SqlAlterTableSchema} with source table to generate new {@link Schema}. */ +public class AlterTableSchemaUtil { + + private final SqlValidator sqlValidator; + private final Function<SqlNode, String> escapeExpression; + private final SchemaResolver schemaResolver; + + AlterTableSchemaUtil( + SqlValidator sqlValidator, + Function<SqlNode, String> escapeExpression, + SchemaResolver schemaResolver) { + this.sqlValidator = sqlValidator; + this.escapeExpression = escapeExpression; + this.schemaResolver = schemaResolver; + } + + public Schema updateSchema(SqlAlterTableSchema alterTableSchema, Schema unresolvedSchema) { + AlterSchemaStrategy strategy = computeAlterSchemaStrategy(alterTableSchema); + SchemaConverter converter = + strategy == AlterSchemaStrategy.ADD + ? new AddSchemaConverter( + unresolvedSchema, + (FlinkTypeFactory) sqlValidator.getTypeFactory(), + sqlValidator, + escapeExpression, + schemaResolver) + : new ModifySchemaConverter( + unresolvedSchema, + (FlinkTypeFactory) sqlValidator.getTypeFactory(), + sqlValidator, + escapeExpression, + schemaResolver); + converter.updateColumn(alterTableSchema.getColumnPositions().getList()); + alterTableSchema.getWatermark().ifPresent(converter::updateWatermark); + alterTableSchema.getFullConstraint().ifPresent(converter::updatePrimaryKey); + return converter.convert(); + } + + private abstract static class SchemaConverter { + static final String EX_MSG_PREFIX = "Failed to execute ALTER TABLE statement.\n"; + + List<String> sortedColumnNames = new ArrayList<>(); + Set<String> alterColNames = new HashSet<>(); + + Map<String, Schema.UnresolvedColumn> columns = new HashMap<>(); + Map<String, Schema.UnresolvedWatermarkSpec> watermarkSpecs = new HashMap<>(); + @Nullable Schema.UnresolvedPrimaryKey primaryKey = null; + Function<SqlNode, String> escapeExpressions; + FlinkTypeFactory typeFactory; + SqlValidator sqlValidator; + SchemaResolver schemaResolver; + + SchemaConverter( + Schema unresolvedSchema, + FlinkTypeFactory typeFactory, + SqlValidator sqlValidator, + Function<SqlNode, String> escapeExpressions, + SchemaResolver schemaResolver) { + this.typeFactory = typeFactory; + this.sqlValidator = sqlValidator; + this.escapeExpressions = escapeExpressions; + this.schemaResolver = schemaResolver; + populateColumnsFromSourceTable(unresolvedSchema); + populatePrimaryKeyFromSourceTable(unresolvedSchema); + populateWatermarkFromSourceTable(unresolvedSchema); + } + + private void populateColumnsFromSourceTable(Schema unresolvedSchema) { + unresolvedSchema + .getColumns() + .forEach( + column -> { + String name = column.getName(); + sortedColumnNames.add(name); + columns.put(name, column); + }); + } + + private void populatePrimaryKeyFromSourceTable(Schema sourceSchema) { + if (sourceSchema.getPrimaryKey().isPresent()) { + primaryKey = sourceSchema.getPrimaryKey().get(); + } + } + + private void populateWatermarkFromSourceTable(Schema sourceSchema) { + for (Schema.UnresolvedWatermarkSpec sourceWatermarkSpec : + sourceSchema.getWatermarkSpecs()) { + watermarkSpecs.put(sourceWatermarkSpec.getColumnName(), sourceWatermarkSpec); + } + } + + private void updateColumn(List<SqlNode> alterColumnPositions) { + collectColumnPosition(alterColumnPositions); + for (SqlNode alterColumnPos : alterColumnPositions) { + SqlTableColumn alterColumn = ((SqlTableColumnPosition) alterColumnPos).getColumn(); + if (alterColumn instanceof SqlTableColumn.SqlComputedColumn) { + convertComputedColumn((SqlTableColumn.SqlComputedColumn) alterColumn); + } else { + convertNonComputedColumn(alterColumn); + } + } + } + + private void updatePrimaryKey(SqlTableConstraint alterPrimaryKey) { + checkPrimaryKeyExists(); + List<String> primaryKeyColumns = Arrays.asList(alterPrimaryKey.getColumnNames()); + if (alterColNames.isEmpty()) { + primaryKeyColumns.forEach(this::updatePrimaryKeyNullability); + } + primaryKey = + new Schema.UnresolvedPrimaryKey( + alterPrimaryKey + .getConstraintName() + .orElseGet( + () -> + primaryKeyColumns.stream() + .collect( + Collectors.joining( + "_", "PK_", ""))), + primaryKeyColumns); + } + + private void updatePrimaryKeyNullability(String columnName) { + Schema.UnresolvedColumn column = columns.get(columnName); + if (column instanceof Schema.UnresolvedPhysicalColumn) { + AbstractDataType<?> originType = + ((Schema.UnresolvedPhysicalColumn) column).getDataType(); + columns.put( + columnName, + new Schema.UnresolvedPhysicalColumn( + columnName, + originType.notNull(), + column.getComment().orElse(null))); + } + } + + private void updateWatermark(SqlWatermark alterWatermarkSpec) { + checkWatermarkExists(); + SqlIdentifier eventTimeColumnName = alterWatermarkSpec.getEventTimeColumnName(); + String rowtimeField = String.join(".", eventTimeColumnName.names); Review Comment: > Can we define watermark on the nest field now? No. > Why not throw exception to notify users? This is the point I've struggled with. If users create a table specifying the nested column as a watermark, `SchemaResolver` will check this and throw the exception. E.g. ```sql Flink SQL> create table T (f0 row<ts timestamp(3), order_id bigint, user_id bigint>, watermark for f0.ts as f0.ts) with ('connector' = 'datagen'); [ERROR] Could not execute SQL statement. Reason: org.apache.flink.table.api.ValidationException: Invalid column name 'f0.ts' for rowtime attribute in watermark declaration. Available columns are: [f0] ``` So I left this check to `SchemaResolver`. I'm fine to throw an exception here. -- 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]
