LadyForest commented on code in PR #20652:
URL: https://github.com/apache/flink/pull/20652#discussion_r1037875171


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/AlterTableSchemaUtil.java:
##########
@@ -0,0 +1,783 @@
+/*
+ * 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.ResolvedCatalogTable;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.SqlCallExpression;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.parse.CalciteParser;
+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.runtime.CalciteContextException;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+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 SqlParser.Config parserConfig;
+    private final SqlValidator sqlValidator;
+    private final Consumer<SqlTableConstraint> validateTableConstraint;
+    private final Function<SqlNode, String> escapeExpression;
+
+    AlterTableSchemaUtil(
+            SqlParser.Config parserConfig,
+            SqlValidator sqlValidator,
+            Function<SqlNode, String> escapeExpression,
+            Consumer<SqlTableConstraint> validateTableConstraint) {
+        this.parserConfig = parserConfig;
+        this.sqlValidator = sqlValidator;
+        this.validateTableConstraint = validateTableConstraint;
+        this.escapeExpression = escapeExpression;
+    }
+
+    public Schema convertSchema(
+            SqlAlterTableSchema alterTableSchema, ResolvedCatalogTable 
originalTable) {
+        UnresolvedSchemaBuilder builder =
+                new UnresolvedSchemaBuilder(
+                        originalTable,
+                        parserConfig,
+                        (FlinkTypeFactory) sqlValidator.getTypeFactory(),
+                        sqlValidator,
+                        validateTableConstraint,
+                        escapeExpression);
+        AlterSchemaStrategy strategy = 
computeAlterSchemaStrategy(alterTableSchema);
+        List<SqlNode> columnPositions = 
alterTableSchema.getColumnPositions().getList();
+        builder.addOrModifyColumns(strategy, columnPositions);
+        alterTableSchema
+                .getWatermark()
+                .ifPresent(sqlWatermark -> 
builder.addOrModifyWatermarks(strategy, sqlWatermark));
+        alterTableSchema
+                .getFullConstraint()
+                .ifPresent(
+                        (pk) ->
+                                builder.addOrModifyPrimaryKey(
+                                        strategy, pk, 
columnPositions.isEmpty()));
+        if (strategy == AlterSchemaStrategy.MODIFY) {
+            builder.verifyComputedColumn();
+            builder.verifyWatermark();
+            builder.verifyPrimaryKey();
+        }
+        return builder.build();
+    }
+
+    private static class UnresolvedSchemaBuilder {
+
+        List<String> sortedColumnNames = new ArrayList<>();
+        Map<String, Schema.UnresolvedColumn> columns = new HashMap<>();
+        Map<String, Schema.UnresolvedWatermarkSpec> watermarkSpecs = new 
HashMap<>();
+        @Nullable Schema.UnresolvedPrimaryKey primaryKey = null;
+
+        // Intermediate state
+        Map<String, RelDataType> physicalFieldNamesToTypes = new HashMap<>();
+        Map<String, RelDataType> metadataFieldNamesToTypes = new HashMap<>();
+        Map<String, RelDataType> computedFieldNamesToTypes = new HashMap<>();
+
+        Map<String, RelDataType> modifiedFieldNamesToOriginTypes = new 
HashMap<>();
+        Map<String, String> computedFieldNamesToExpressions = new HashMap<>();
+
+        SqlParser.Config parserConfig;
+        Function<SqlNode, String> escapeExpressions;
+        FlinkTypeFactory typeFactory;
+        SqlValidator sqlValidator;
+        Consumer<SqlTableConstraint> validateTableConstraint;
+
+        UnresolvedSchemaBuilder(
+                ResolvedCatalogTable sourceTable,
+                SqlParser.Config parserConfig,
+                FlinkTypeFactory typeFactory,
+                SqlValidator sqlValidator,
+                Consumer<SqlTableConstraint> validateTableConstraint,
+                Function<SqlNode, String> escapeExpressions) {
+            this.parserConfig = parserConfig;
+            this.typeFactory = typeFactory;
+            this.sqlValidator = sqlValidator;
+            this.escapeExpressions = escapeExpressions;
+            this.validateTableConstraint = validateTableConstraint;
+            populateColumnsFromSourceTable(sourceTable);
+            
populatePrimaryKeyFromSourceTable(sourceTable.getUnresolvedSchema());
+            
populateWatermarksFromSourceTable(sourceTable.getUnresolvedSchema());
+        }
+
+        private void populateColumnsFromSourceTable(ResolvedCatalogTable 
sourceTable) {
+            List<DataType> types = 
sourceTable.getResolvedSchema().getColumnDataTypes();
+            List<Schema.UnresolvedColumn> sourceColumns =
+                    sourceTable.getUnresolvedSchema().getColumns();
+            for (int i = 0; i < sourceColumns.size(); i++) {
+                Schema.UnresolvedColumn column = sourceColumns.get(i);
+                String columnName = column.getName();
+                sortedColumnNames.add(columnName);
+                columns.put(columnName, column);
+                RelDataType type =
+                        
typeFactory.createFieldTypeFromLogicalType(types.get(i).getLogicalType());
+                if (column instanceof Schema.UnresolvedPhysicalColumn) {
+                    physicalFieldNamesToTypes.put(columnName, type);
+                } else if (column instanceof Schema.UnresolvedComputedColumn) {
+                    computedFieldNamesToTypes.put(columnName, type);
+                    computedFieldNamesToExpressions.put(
+                            columnName,
+                            ((SqlCallExpression)
+                                            ((Schema.UnresolvedComputedColumn) 
column)
+                                                    .getExpression())
+                                    .getSqlExpression());
+                } else if (column instanceof Schema.UnresolvedMetadataColumn) {
+                    metadataFieldNamesToTypes.put(columnName, type);
+                }
+            }
+        }
+
+        private void populatePrimaryKeyFromSourceTable(Schema sourceSchema) {
+            if (sourceSchema.getPrimaryKey().isPresent()) {
+                primaryKey = sourceSchema.getPrimaryKey().get();
+            }
+        }
+
+        private void populateWatermarksFromSourceTable(Schema sourceSchema) {
+            for (Schema.UnresolvedWatermarkSpec sourceWatermarkSpec :
+                    sourceSchema.getWatermarkSpecs()) {
+                watermarkSpecs.put(sourceWatermarkSpec.getColumnName(), 
sourceWatermarkSpec);
+            }
+        }
+
+        private void addOrModifyColumns(
+                AlterSchemaStrategy strategy, List<SqlNode> 
alterColumnPositions) {
+            collectColumnPosition(strategy, alterColumnPositions);
+            for (SqlNode alterColumnPos : alterColumnPositions) {
+                SqlTableColumn alterColumn = ((SqlTableColumnPosition) 
alterColumnPos).getColumn();
+                if (strategy == AlterSchemaStrategy.MODIFY) {
+                    // column type might be changed
+                    String columnName = alterColumn.getName().getSimple();
+                    RelDataType dataType = 
physicalFieldNamesToTypes.remove(columnName);
+                    if (dataType == null) {
+                        dataType = 
metadataFieldNamesToTypes.remove(columnName);
+                    }
+                    if (dataType == null) {
+                        dataType = 
computedFieldNamesToTypes.remove(columnName);
+                    }
+                    computedFieldNamesToExpressions.remove(columnName);
+                    modifiedFieldNamesToOriginTypes.put(columnName, dataType);
+                }
+                if (alterColumn instanceof SqlTableColumn.SqlComputedColumn) {
+                    convertComputedColumn((SqlTableColumn.SqlComputedColumn) 
alterColumn, strategy);
+                } else {
+                    convertNonComputedColumn(alterColumn);
+                }
+            }
+        }
+
+        private void addOrModifyPrimaryKey(
+                AlterSchemaStrategy strategy,
+                SqlTableConstraint alterPrimaryKey,
+                boolean withoutAlterColumn) {
+            validateTableConstraint.accept(alterPrimaryKey);
+            if (strategy == AlterSchemaStrategy.ADD && primaryKey != null) {
+                throw new ValidationException(
+                        String.format(
+                                "Could not create primary key because the base 
table has already defined the primary key constraint %s. You might "
+                                        + "want to drop it before adding a new 
one.",
+                                primaryKey.getColumnNames().stream()
+                                        .collect(Collectors.joining("`, `", 
"[`", "`]"))));
+            } else if (strategy == AlterSchemaStrategy.MODIFY && primaryKey == 
null) {
+                throw new ValidationException(
+                        "Could not modify primary key because the base table 
does not define any primary key. You might "
+                                + "want to add a new one.");
+            }
+            List<String> primaryKeyColumns = new ArrayList<>();
+            for (SqlNode primaryKeyNode : alterPrimaryKey.getColumns()) {
+                String primaryKey = ((SqlIdentifier) 
primaryKeyNode).getSimple();
+                if (!columns.containsKey(primaryKey)) {
+                    throw new ValidationException(
+                            String.format(
+                                    "Could not %s primary key because the 
specified column `%s` is not defined in the table schema at %s.",
+                                    strategy == AlterSchemaStrategy.ADD ? 
"create" : "modify",
+                                    primaryKey,
+                                    primaryKeyNode.getParserPosition()));
+                }
+                if (!(columns.get(primaryKey) instanceof 
Schema.UnresolvedPhysicalColumn)) {
+                    throw new ValidationException(
+                            String.format(
+                                    "Could not %s the primary key with column 
`%s` at %s.\n"
+                                            + "A primary key constraint must 
be declared on physical columns.",
+                                    strategy == AlterSchemaStrategy.ADD ? 
"create" : "modify",
+                                    primaryKey,
+                                    primaryKeyNode.getParserPosition()));
+                }
+                primaryKeyColumns.add(primaryKey);
+            }
+            if (withoutAlterColumn) {
+                // a single add/modify constraint changes the nullability of 
columns implicitly
+                for (String primaryKeyColumn : primaryKeyColumns) {
+                    fixPrimaryKeyNullability(primaryKeyColumn);
+                }
+            }
+            primaryKey =
+                    primaryKeyNamed(
+                            alterPrimaryKey
+                                    .getConstraintName()
+                                    .orElseGet(
+                                            () ->
+                                                    primaryKeyColumns.stream()
+                                                            .collect(
+                                                                    
Collectors.joining(
+                                                                            
"_", "PK_", ""))),
+                            primaryKeyColumns);
+        }
+
+        private void fixPrimaryKeyNullability(String columnName) {
+            DataType alterDataType =
+                    fromLogicalToDataType(
+                            toLogicalType(
+                                    typeFactory.createTypeWithNullability(
+                                            
physicalFieldNamesToTypes.get(columnName), false)));
+            Schema.UnresolvedColumn column = columns.remove(columnName);
+            columns.put(
+                    columnName,
+                    column(columnName, alterDataType, 
column.getComment().orElse(null)));
+        }
+
+        private void verifyPrimaryKey() {
+            if (primaryKey != null) {
+                for (String columnName : primaryKey.getColumnNames()) {
+                    boolean computedColumn = 
computedFieldNamesToTypes.containsKey(columnName);
+                    boolean metadataColumn = 
metadataFieldNamesToTypes.containsKey(columnName);
+                    if (computedColumn || metadataColumn) {
+                        throw new ValidationException(
+                                String.format(
+                                        "Could not modify column `%s` from 
type '%s' to %s column type '%s' "
+                                                + "because the base table 
exists the primary key constraint %s.",
+                                        columnName,
+                                        modifiedFieldNamesToOriginTypes
+                                                .get(columnName)
+                                                .getFullTypeString(),
+                                        computedColumn ? "computed" : 
"metadata",
+                                        computedColumn
+                                                ? computedFieldNamesToTypes
+                                                        .get(columnName)
+                                                        .getFullTypeString()
+                                                : metadataFieldNamesToTypes
+                                                        .get(columnName)
+                                                        .getFullTypeString(),
+                                        primaryKey.getColumnNames().stream()
+                                                
.collect(Collectors.joining("`, ", "[`", "`]"))));
+                    } else {
+                        fixPrimaryKeyNullability(columnName);
+                    }
+                }
+            }
+        }
+
+        private void addOrModifyWatermarks(
+                AlterSchemaStrategy strategy, SqlWatermark alterWatermarkSpec) 
{
+            SqlIdentifier eventTimeColumnName = 
alterWatermarkSpec.getEventTimeColumnName();
+            if (!eventTimeColumnName.isSimple()) {
+                throw new ValidationException(
+                        String.format(
+                                "Could not %s watermark spec because the 
nested rowtime attribute field %s cannot define a watermark.",
+                                strategy == AlterSchemaStrategy.ADD ? "create" 
: "modify",
+                                eventTimeColumnName.names.stream()
+                                        .collect(Collectors.joining("`.`", 
"`", "`"))));
+            }
+            String rowtimeField = eventTimeColumnName.getSimple();
+            SqlNode validated =
+                    verifyWatermark(
+                            strategy, rowtimeField, 
alterWatermarkSpec.getWatermarkStrategy());
+            watermarkSpecs.clear();
+            watermarkSpecs.put(
+                    rowtimeField,
+                    watermarkSpec(
+                            rowtimeField,
+                            new 
SqlCallExpression(escapeExpressions.apply(validated))));
+        }
+
+        private SqlNode verifyWatermark(
+                AlterSchemaStrategy strategy, String rowtimeField, SqlNode 
watermarkStrategy) {
+            Map<String, RelDataType> nameToTypeMap = new HashMap<>();
+            nameToTypeMap.putAll(physicalFieldNamesToTypes);
+            nameToTypeMap.putAll(metadataFieldNamesToTypes);
+            nameToTypeMap.putAll(computedFieldNamesToTypes);
+            verifyRowtimeAttribute(strategy, rowtimeField, nameToTypeMap);
+
+            // this will validate and expand function identifiers
+            SqlNode validated =
+                    
sqlValidator.validateParameterizedExpression(watermarkStrategy, nameToTypeMap);
+
+            String fullTypeString =
+                    
sqlValidator.getValidatedNodeType(validated).getFullTypeString();
+            // ignore the nullability comparison
+            if (!fullTypeString.contains("TIMESTAMP(3)")) {
+                throw new ValidationException(
+                        String.format(
+                                "Could not %s watermark spec because the 
watermark spec %s produces the type '%s', but 'TIMESTAMP(3)' is required.",
+                                strategy == AlterSchemaStrategy.ADD ? "create" 
: "modify",
+                                escapeExpressions.apply(watermarkStrategy),
+                                fullTypeString));
+            }
+            return validated;
+        }
+
+        private void verifyWatermark() {
+            if (!watermarkSpecs.isEmpty()) {
+                Map<String, RelDataType> nameToTypeMap = new HashMap<>();
+                nameToTypeMap.putAll(physicalFieldNamesToTypes);
+                nameToTypeMap.putAll(metadataFieldNamesToTypes);
+                nameToTypeMap.putAll(computedFieldNamesToTypes);
+                for (Map.Entry<String, Schema.UnresolvedWatermarkSpec> entry :
+                        watermarkSpecs.entrySet()) {
+                    String columnName = entry.getKey();
+                    String watermarkStrategy =
+                            ((SqlCallExpression) 
entry.getValue().getWatermarkExpression())
+                                    .getSqlExpression();
+                    verifyRowtimeAttribute(AlterSchemaStrategy.MODIFY, 
columnName, nameToTypeMap);
+                    try {
+                        CalciteParser parser = new CalciteParser(parserConfig);
+                        SqlNode validated =
+                                sqlValidator.validateParameterizedExpression(
+                                        
parser.parseExpression(watermarkStrategy), nameToTypeMap);
+
+                        String fullTypeString =
+                                
sqlValidator.getValidatedNodeType(validated).getFullTypeString();
+                        // ignore the nullability comparison
+                        if (!fullTypeString.contains("TIMESTAMP(3)")) {
+                            // the type derived from strategy expression is 
invalid
+                            // due to the rowtime field's type change
+                            throw new ValidationException(
+                                    String.format(
+                                            "Could not modify column `%s` from 
type '%s' to '%s' because the base table exists watermark spec [%s], "
+                                                    + "which produces type 
'%s', but 'TIMESTAMP(3)' is required.",
+                                            columnName,
+                                            modifiedFieldNamesToOriginTypes
+                                                    .get(columnName)
+                                                    .getFullTypeString(),
+                                            
nameToTypeMap.get(columnName).getFullTypeString(),
+                                            watermarkStrategy,
+                                            fullTypeString));
+                        }
+                    } catch (Throwable t) {
+                        // the strategy expression is invalid
+                        // due to the rowtime field's type change
+                        String exMsg =
+                                String.format(
+                                        "Could not modify column `%s` from 
type '%s' to '%s' because the base table exists watermark spec [%s].",
+                                        columnName,
+                                        modifiedFieldNamesToOriginTypes
+                                                .get(columnName)
+                                                .getFullTypeString(),
+                                        
nameToTypeMap.get(columnName).getFullTypeString(),
+                                        watermarkStrategy);
+                        throw new ValidationException(exMsg, t);
+                    }
+                }
+            }
+        }
+
+        private void convertNonComputedColumn(SqlTableColumn column) {
+            boolean isPhysical = column instanceof 
SqlTableColumn.SqlRegularColumn;
+            String name = column.getName().getSimple();
+            SqlDataTypeSpec typeSpec =
+                    isPhysical
+                            ? ((SqlTableColumn.SqlRegularColumn) 
column).getType()
+                            : ((SqlTableColumn.SqlMetadataColumn) 
column).getType();
+            RelDataType relType =
+                    typeSpec.deriveType(
+                            sqlValidator, typeSpec.getNullable() == null || 
typeSpec.getNullable());
+            DataType dataType = fromLogicalToDataType(toLogicalType(relType));
+
+            if (isPhysical) {
+                physicalFieldNamesToTypes.put(name, relType);
+            } else {
+                metadataFieldNamesToTypes.put(name, relType);
+            }
+            Schema.UnresolvedColumn newColumn =
+                    isPhysical
+                            ? column(name, dataType, getComment(column))
+                            : columnByMetadata(
+                                    name,
+                                    dataType,
+                                    ((SqlTableColumn.SqlMetadataColumn) column)
+                                            .getMetadataAlias()
+                                            .orElse(null),
+                                    ((SqlTableColumn.SqlMetadataColumn) 
column).isVirtual(),
+                                    getComment(column));
+
+            columns.put(name, newColumn);
+        }
+
+        private void convertComputedColumn(
+                SqlTableColumn.SqlComputedColumn column, AlterSchemaStrategy 
strategy) {
+            String columnName = column.getName().getSimple();
+
+            Map<String, RelDataType> accessibleFieldNamesToTypes = new 
HashMap<>();
+            accessibleFieldNamesToTypes.putAll(physicalFieldNamesToTypes);
+            accessibleFieldNamesToTypes.putAll(metadataFieldNamesToTypes);
+
+            try {
+                final SqlNode validatedExpr =
+                        sqlValidator.validateParameterizedExpression(
+                                column.getExpr(), accessibleFieldNamesToTypes);
+                String escapedExpr = escapeExpressions.apply(validatedExpr);
+                Schema.UnresolvedColumn newColumn =
+                        columnByExpression(columnName, escapedExpr, 
getComment(column));
+                computedFieldNamesToTypes.put(
+                        columnName, 
sqlValidator.getValidatedNodeType(validatedExpr));
+                computedFieldNamesToExpressions.put(columnName, escapedExpr);
+                columns.put(columnName, newColumn);
+            } catch (Exception e) {
+                String exMsg =
+                        String.format(
+                                "Could not %s computed column `%s` AS %s.\n%s",
+                                strategy == AlterSchemaStrategy.ADD ? "create" 
: "modify",
+                                columnName,
+                                escapeExpressions.apply(column.getExpr()),
+                                e.getMessage());
+                if (e instanceof CalciteContextException
+                        && e.getMessage().contains("Unknown identifier")) {
+                    exMsg =
+                            String.format(
+                                    "%s\nComputed column can only be derived 
from existed physical or metadata column.",
+                                    exMsg);
+                }
+                throw new ValidationException(exMsg);
+            }
+        }
+
+        private void verifyComputedColumn() {
+            Map<String, RelDataType> accessibleFieldNamesToTypes = new 
HashMap<>();
+            accessibleFieldNamesToTypes.putAll(physicalFieldNamesToTypes);
+            accessibleFieldNamesToTypes.putAll(metadataFieldNamesToTypes);
+            for (Map.Entry<String, String> entry : 
computedFieldNamesToExpressions.entrySet()) {
+                CalciteParser parser = new CalciteParser(parserConfig);
+                String columnName = entry.getKey();
+                SqlNode expr = parser.parseExpression(entry.getValue());
+                try {
+                    sqlValidator.validateParameterizedExpression(expr, 
accessibleFieldNamesToTypes);
+                } catch (Throwable e) {

Review Comment:
   The reason why catch `Throwable` instead of `Exception` (or, to be specific, 
`CalciteContextException`) is because 
`sqlValidator#validateParameterizedExpression` might throw 
   - NPE, E.g. `e ROW<f0 STRING, f1 INT, f2 ROW<DOUBLE, ARRAY<FLOAT>>>, f AS 
e.f1 + e.f2.f0`, and `ALTER TABLE MODIFY e STRING`
   - `AssertionError` when `inferReturnType`



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