You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/12/12 11:50:05 UTC

[GitHub] [flink] fsk119 commented on a diff in pull request #20652: [FLINK-22315][table] Support add/modify column/constraint/watermark for ALTER TABLE statement

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


##########
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);

Review Comment:
   ???



##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/SqlConstraintValidator.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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;
+
+import org.apache.flink.sql.parser.ddl.SqlTableColumn;
+import org.apache.flink.sql.parser.ddl.constraint.SqlTableConstraint;
+import org.apache.flink.sql.parser.error.SqlValidateException;
+
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Util to validate {@link SqlTableConstraint}. */
+public class SqlConstraintValidator {
+
+    /** Returns the column constraints plus the table constraints. */
+    public static List<SqlTableConstraint> getFullConstraints(
+            List<SqlTableConstraint> tableConstraints, SqlNodeList columnList) {
+        List<SqlTableConstraint> ret = new ArrayList<>();
+        columnList.forEach(
+                column -> {
+                    SqlTableColumn tableColumn = (SqlTableColumn) column;
+                    if (tableColumn instanceof SqlTableColumn.SqlRegularColumn) {
+                        SqlTableColumn.SqlRegularColumn regularColumn =
+                                (SqlTableColumn.SqlRegularColumn) tableColumn;
+                        regularColumn.getConstraint().map(ret::add);
+                    }
+                });
+        ret.addAll(tableConstraints);
+        return ret;
+    }
+
+    /** Check duplicate constraints and change the nullability of primary key columns. */
+    public static void validate(List<SqlTableConstraint> tableConstraints, SqlNodeList columnList)

Review Comment:
   If the method changes the inputs, I think we should add the info into the method name. So others can understand directly and don't need to read the java doc.  
   
   ```
   validateAndChangeColumnNullablitiy
   ```
   
   Otherwise, the method should return a new column list with type changed.



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

Review Comment:
   Can we rename to `AlterSchemaConverter`? In the most cases, the util class only exposes static method.



##########
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) {

Review Comment:
   Rename to `applySchemaChange` ?



##########
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);
+            watermarkSpecs.clear();
+            watermarkSpecs.put(
+                    rowtimeField,
+                    new Schema.UnresolvedWatermarkSpec(
+                            rowtimeField,
+                            new SqlCallExpression(
+                                    escapeExpressions.apply(
+                                            alterWatermarkSpec.getWatermarkStrategy()))));
+        }
+
+        private void convertNonComputedColumn(SqlTableColumn column) {
+            boolean isPhysical = column instanceof SqlTableColumn.SqlRegularColumn;
+            Schema.UnresolvedColumn newColumn =
+                    isPhysical
+                            ? convertPhysicalColumn((SqlTableColumn.SqlRegularColumn) column)
+                            : convertMetadataColumn((SqlTableColumn.SqlMetadataColumn) column);
+            columns.put(column.getName().getSimple(), newColumn);
+        }
+
+        Schema.UnresolvedPhysicalColumn convertPhysicalColumn(
+                SqlTableColumn.SqlRegularColumn physicalColumn) {
+            DataType dataType = getDataType(physicalColumn.getType());
+            return new Schema.UnresolvedPhysicalColumn(
+                    physicalColumn.getName().getSimple(), dataType, getComment(physicalColumn));
+        }
+
+        private Schema.UnresolvedMetadataColumn convertMetadataColumn(
+                SqlTableColumn.SqlMetadataColumn metadataColumn) {
+            DataType dataType = getDataType(metadataColumn.getType());
+            return new Schema.UnresolvedMetadataColumn(
+                    metadataColumn.getName().getSimple(),
+                    dataType,
+                    metadataColumn.getMetadataAlias().orElse(null),
+                    metadataColumn.isVirtual(),
+                    getComment(metadataColumn));
+        }
+
+        private void convertComputedColumn(SqlTableColumn.SqlComputedColumn column) {
+            String columnName = column.getName().getSimple();
+            Schema.UnresolvedColumn newColumn =
+                    new Schema.UnresolvedComputedColumn(
+                            columnName,
+                            new SqlCallExpression(escapeExpressions.apply(column.getExpr())),
+                            getComment(column));
+            columns.put(columnName, newColumn);
+        }
+
+        private DataType getDataType(SqlDataTypeSpec typeSpec) {
+            RelDataType relType =
+                    typeSpec.deriveType(
+                            sqlValidator, typeSpec.getNullable() == null || typeSpec.getNullable());
+            return fromLogicalToDataType(toLogicalType(relType));
+        }
+
+        @Nullable
+        String getComment(SqlTableColumn column) {
+            return column.getComment()
+                    .map(SqlCharStringLiteral.class::cast)
+                    .map(c -> c.getValueAs(String.class))
+                    .orElse(null);
+        }
+
+        private void collectColumnPosition(List<SqlNode> alterColumns) {
+            for (SqlNode alterColumn : alterColumns) {
+                SqlTableColumnPosition columnPosition = (SqlTableColumnPosition) alterColumn;
+                SqlTableColumn column = columnPosition.getColumn();
+                if (!column.getName().isSimple()) {
+                    throw new UnsupportedOperationException(
+                            String.format(
+                                    "%sAlter nested row type is not supported yet.",
+                                    EX_MSG_PREFIX));
+                }
+                String columnName = column.getName().getSimple();
+                boolean first = columnPosition.isFirstColumn();
+                boolean after = columnPosition.isAfterReferencedColumn();

Review Comment:
   nit: Can we inline this?



##########
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);
+            watermarkSpecs.clear();
+            watermarkSpecs.put(
+                    rowtimeField,
+                    new Schema.UnresolvedWatermarkSpec(
+                            rowtimeField,
+                            new SqlCallExpression(
+                                    escapeExpressions.apply(
+                                            alterWatermarkSpec.getWatermarkStrategy()))));
+        }
+
+        private void convertNonComputedColumn(SqlTableColumn column) {
+            boolean isPhysical = column instanceof SqlTableColumn.SqlRegularColumn;
+            Schema.UnresolvedColumn newColumn =
+                    isPhysical
+                            ? convertPhysicalColumn((SqlTableColumn.SqlRegularColumn) column)
+                            : convertMetadataColumn((SqlTableColumn.SqlMetadataColumn) column);
+            columns.put(column.getName().getSimple(), newColumn);
+        }
+
+        Schema.UnresolvedPhysicalColumn convertPhysicalColumn(
+                SqlTableColumn.SqlRegularColumn physicalColumn) {
+            DataType dataType = getDataType(physicalColumn.getType());
+            return new Schema.UnresolvedPhysicalColumn(
+                    physicalColumn.getName().getSimple(), dataType, getComment(physicalColumn));
+        }
+
+        private Schema.UnresolvedMetadataColumn convertMetadataColumn(
+                SqlTableColumn.SqlMetadataColumn metadataColumn) {
+            DataType dataType = getDataType(metadataColumn.getType());
+            return new Schema.UnresolvedMetadataColumn(
+                    metadataColumn.getName().getSimple(),
+                    dataType,
+                    metadataColumn.getMetadataAlias().orElse(null),
+                    metadataColumn.isVirtual(),
+                    getComment(metadataColumn));
+        }
+
+        private void convertComputedColumn(SqlTableColumn.SqlComputedColumn column) {
+            String columnName = column.getName().getSimple();
+            Schema.UnresolvedColumn newColumn =
+                    new Schema.UnresolvedComputedColumn(
+                            columnName,
+                            new SqlCallExpression(escapeExpressions.apply(column.getExpr())),
+                            getComment(column));
+            columns.put(columnName, newColumn);
+        }
+
+        private DataType getDataType(SqlDataTypeSpec typeSpec) {
+            RelDataType relType =
+                    typeSpec.deriveType(
+                            sqlValidator, typeSpec.getNullable() == null || typeSpec.getNullable());
+            return fromLogicalToDataType(toLogicalType(relType));
+        }
+
+        @Nullable
+        String getComment(SqlTableColumn column) {
+            return column.getComment()
+                    .map(SqlCharStringLiteral.class::cast)
+                    .map(c -> c.getValueAs(String.class))
+                    .orElse(null);
+        }
+
+        private void collectColumnPosition(List<SqlNode> alterColumns) {
+            for (SqlNode alterColumn : alterColumns) {
+                SqlTableColumnPosition columnPosition = (SqlTableColumnPosition) alterColumn;
+                SqlTableColumn column = columnPosition.getColumn();
+                if (!column.getName().isSimple()) {
+                    throw new UnsupportedOperationException(
+                            String.format(
+                                    "%sAlter nested row type is not supported yet.",
+                                    EX_MSG_PREFIX));
+                }
+                String columnName = column.getName().getSimple();
+                boolean first = columnPosition.isFirstColumn();
+                boolean after = columnPosition.isAfterReferencedColumn();
+                if (!alterColNames.add(columnName)) {
+                    throw new ValidationException(
+                            String.format(
+                                    "%sEncounter duplicate column `%s`.",
+                                    EX_MSG_PREFIX, columnName));
+                }
+                checkColumnExists(columnName);
+                if (first) {
+                    addFirst(columnName);
+                } else if (after) {
+                    addAfter(columnName, getReferencedColumn(columnPosition));
+                } else {
+                    addLast(columnName);
+                }
+            }
+        }
+
+        private String getReferencedColumn(SqlTableColumnPosition columnPosition) {
+            SqlIdentifier referencedIdent = columnPosition.getAfterReferencedColumn();
+            Preconditions.checkNotNull(
+                    referencedIdent,
+                    String.format("%sCould not refer to a null column", EX_MSG_PREFIX));
+            if (!referencedIdent.isSimple()) {
+                throw new UnsupportedOperationException(
+                        String.format(
+                                "%sAlter nested row type is not supported yet.", EX_MSG_PREFIX));
+            }
+            String referencedName = referencedIdent.getSimple();
+            if (!sortedColumnNames.contains(referencedName)) {
+                throw new ValidationException(
+                        String.format(
+                                "%sReferenced column `%s` by 'AFTER' does not exist in the table.",
+                                EX_MSG_PREFIX, referencedName));
+            }
+            return referencedName;
+        }
+
+        private Schema convert() {
+            List<Schema.UnresolvedColumn> newColumns = new ArrayList<>();
+            for (String column : sortedColumnNames) {
+                newColumns.add(columns.get(column));
+            }
+            Schema.Builder resultBuilder = Schema.newBuilder().fromColumns(newColumns);
+            if (primaryKey != null) {
+                String constraintName = primaryKey.getConstraintName();
+                List<String> pkColumns = primaryKey.getColumnNames();
+                if (constraintName != null) {
+                    resultBuilder.primaryKeyNamed(constraintName, pkColumns);
+                } else {
+                    resultBuilder.primaryKey(pkColumns);
+                }
+            }
+            watermarkSpecs.forEach(
+                    (k, v) -> resultBuilder.watermark(k, v.getWatermarkExpression()));
+            Schema updatedSchema = resultBuilder.build();
+            try {
+                schemaResolver.resolve(updatedSchema);
+                return updatedSchema;
+            } catch (Exception e) {
+                throw new ValidationException(String.format("%s%s", EX_MSG_PREFIX, e.getMessage()));
+            }
+        }
+
+        abstract void checkColumnExists(String columnName);
+
+        abstract void checkPrimaryKeyExists();
+
+        abstract void checkWatermarkExists();
+
+        abstract void addFirst(String columnName);

Review Comment:
   addFirst is a little confusing to me.. With the modify syntax, it actually moves the column to the first?



##########
flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseConnectorITCase.java:
##########
@@ -152,7 +152,7 @@ public void testTableSourceFieldOrder() {
 
         tEnv.executeSql(
                 "CREATE TABLE hTable ("
-                        + " rowkey INT PRIMARY KEY,"
+                        + " rowkey INT PRIMARY KEY NOT ENFORCED,"

Review Comment:
   Why need to modify this? Is it a break change?



##########
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);
+            watermarkSpecs.clear();
+            watermarkSpecs.put(
+                    rowtimeField,
+                    new Schema.UnresolvedWatermarkSpec(
+                            rowtimeField,
+                            new SqlCallExpression(
+                                    escapeExpressions.apply(
+                                            alterWatermarkSpec.getWatermarkStrategy()))));
+        }
+
+        private void convertNonComputedColumn(SqlTableColumn column) {
+            boolean isPhysical = column instanceof SqlTableColumn.SqlRegularColumn;
+            Schema.UnresolvedColumn newColumn =
+                    isPhysical
+                            ? convertPhysicalColumn((SqlTableColumn.SqlRegularColumn) column)
+                            : convertMetadataColumn((SqlTableColumn.SqlMetadataColumn) column);
+            columns.put(column.getName().getSimple(), newColumn);
+        }
+
+        Schema.UnresolvedPhysicalColumn convertPhysicalColumn(
+                SqlTableColumn.SqlRegularColumn physicalColumn) {
+            DataType dataType = getDataType(physicalColumn.getType());
+            return new Schema.UnresolvedPhysicalColumn(
+                    physicalColumn.getName().getSimple(), dataType, getComment(physicalColumn));
+        }
+
+        private Schema.UnresolvedMetadataColumn convertMetadataColumn(
+                SqlTableColumn.SqlMetadataColumn metadataColumn) {
+            DataType dataType = getDataType(metadataColumn.getType());
+            return new Schema.UnresolvedMetadataColumn(
+                    metadataColumn.getName().getSimple(),
+                    dataType,
+                    metadataColumn.getMetadataAlias().orElse(null),
+                    metadataColumn.isVirtual(),
+                    getComment(metadataColumn));
+        }
+
+        private void convertComputedColumn(SqlTableColumn.SqlComputedColumn column) {
+            String columnName = column.getName().getSimple();
+            Schema.UnresolvedColumn newColumn =
+                    new Schema.UnresolvedComputedColumn(
+                            columnName,
+                            new SqlCallExpression(escapeExpressions.apply(column.getExpr())),
+                            getComment(column));
+            columns.put(columnName, newColumn);
+        }
+
+        private DataType getDataType(SqlDataTypeSpec typeSpec) {
+            RelDataType relType =
+                    typeSpec.deriveType(
+                            sqlValidator, typeSpec.getNullable() == null || typeSpec.getNullable());
+            return fromLogicalToDataType(toLogicalType(relType));
+        }
+
+        @Nullable
+        String getComment(SqlTableColumn column) {
+            return column.getComment()
+                    .map(SqlCharStringLiteral.class::cast)
+                    .map(c -> c.getValueAs(String.class))
+                    .orElse(null);
+        }
+
+        private void collectColumnPosition(List<SqlNode> alterColumns) {
+            for (SqlNode alterColumn : alterColumns) {
+                SqlTableColumnPosition columnPosition = (SqlTableColumnPosition) alterColumn;
+                SqlTableColumn column = columnPosition.getColumn();
+                if (!column.getName().isSimple()) {
+                    throw new UnsupportedOperationException(
+                            String.format(
+                                    "%sAlter nested row type is not supported yet.",
+                                    EX_MSG_PREFIX));
+                }
+                String columnName = column.getName().getSimple();
+                boolean first = columnPosition.isFirstColumn();
+                boolean after = columnPosition.isAfterReferencedColumn();
+                if (!alterColNames.add(columnName)) {
+                    throw new ValidationException(
+                            String.format(
+                                    "%sEncounter duplicate column `%s`.",
+                                    EX_MSG_PREFIX, columnName));
+                }
+                checkColumnExists(columnName);
+                if (first) {
+                    addFirst(columnName);
+                } else if (after) {
+                    addAfter(columnName, getReferencedColumn(columnPosition));
+                } else {
+                    addLast(columnName);
+                }

Review Comment:
   I think we can add a method named `applyColumnPosition` and move all these into the implementation.



##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/SqlConstraintValidator.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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;
+
+import org.apache.flink.sql.parser.ddl.SqlTableColumn;
+import org.apache.flink.sql.parser.ddl.constraint.SqlTableConstraint;
+import org.apache.flink.sql.parser.error.SqlValidateException;
+
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Util to validate {@link SqlTableConstraint}. */
+public class SqlConstraintValidator {
+
+    /** Returns the column constraints plus the table constraints. */
+    public static List<SqlTableConstraint> getFullConstraints(
+            List<SqlTableConstraint> tableConstraints, SqlNodeList columnList) {
+        List<SqlTableConstraint> ret = new ArrayList<>();
+        columnList.forEach(
+                column -> {
+                    SqlTableColumn tableColumn = (SqlTableColumn) column;
+                    if (tableColumn instanceof SqlTableColumn.SqlRegularColumn) {
+                        SqlTableColumn.SqlRegularColumn regularColumn =
+                                (SqlTableColumn.SqlRegularColumn) tableColumn;
+                        regularColumn.getConstraint().map(ret::add);
+                    }
+                });
+        ret.addAll(tableConstraints);
+        return ret;
+    }
+
+    /** Check duplicate constraints and change the nullability of primary key columns. */
+    public static void validate(List<SqlTableConstraint> tableConstraints, SqlNodeList columnList)
+            throws SqlValidateException {
+        List<SqlTableConstraint> fullConstraints = getFullConstraints(tableConstraints, columnList);
+
+        if (fullConstraints.stream().filter(SqlTableConstraint::isPrimaryKey).count() > 1) {
+            throw new SqlValidateException(
+                    fullConstraints.get(1).getParserPosition(), "Duplicate primary key definition");
+        }
+        for (SqlTableConstraint constraint : fullConstraints) {
+            if (constraint.isUnique()) {
+                throw new SqlValidateException(
+                        constraint.getParserPosition(), "UNIQUE constraint is not supported yet");
+            }
+            if (constraint.isEnforced()) {
+                throw new SqlValidateException(
+                        constraint.getParserPosition(),
+                        "Flink doesn't support ENFORCED mode for "
+                                + "PRIMARY KEY constraint. ENFORCED/NOT ENFORCED  controls if the constraint "

Review Comment:
   `ENFORCED/NOT ENFORCED controls` has two spaces here.



##########
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) {
+                    throw new ValidationException(
+                            String.format(
+                                    "Could not modify column %s because the base table exists a computed column `%s` AS %s.",
+                                    printColumnMsg(expr, accessibleFieldNamesToTypes),
+                                    columnName,
+                                    escapeExpressions.apply(expr)),
+                            e);
+                }
+            }
+        }
+
+        private String printColumnMsg(
+                SqlNode expr, Map<String, RelDataType> accessibleFieldNamesToTypes) {
+            ReferredFieldsExtractor extractor = new ReferredFieldsExtractor();
+            expr.accept(extractor);
+            return extractor.referredFields.stream()
+                    .filter(field -> modifiedFieldNamesToOriginTypes.containsKey(field))
+                    .sorted()
+                    .map(
+                            name ->
+                                    String.format(
+                                            "`%s` from type '%s' to %s",
+                                            name,
+                                            modifiedFieldNamesToOriginTypes
+                                                    .get(name)
+                                                    .getFullTypeString(),
+                                            accessibleFieldNamesToTypes.containsKey(name)
+                                                    ? String.format(
+                                                            "'%s'",
+                                                            accessibleFieldNamesToTypes
+                                                                    .get(name)
+                                                                    .getFullTypeString())
+                                                    : computedFieldNamesToExpressions.get(name)))
+                    .collect(Collectors.joining(", "));
+        }
+
+        private void verifyRowtimeAttribute(
+                AlterSchemaStrategy strategy,
+                String rowtimeField,
+                Map<String, RelDataType> allFieldsTypes) {
+            if (!watermarkSpecs.isEmpty() && strategy == AlterSchemaStrategy.ADD) {
+                throw new ValidationException(
+                        String.format(
+                                "Could not create watermark spec because the base table exists a watermark for column %s. You "
+                                        + "might want to drop it before adding a new one.",
+                                watermarkSpecs.keySet().stream()
+                                        .collect(Collectors.joining("`.`", "`", "`"))));
+            } else if (watermarkSpecs.isEmpty() && strategy == AlterSchemaStrategy.MODIFY) {
+                throw new ValidationException(
+                        "Could not modify watermark spec because the base table does not exist any watermark. You might want to add a new one.");
+            }
+
+            if (!allFieldsTypes.containsKey(rowtimeField)) {
+                throw new ValidationException(
+                        String.format(
+                                "Could not %s watermark spec because the rowtime attribute field `%s` is not defined in the table schema.\n"
+                                        + "Available fields: %s",
+                                strategy == AlterSchemaStrategy.ADD ? "create" : "modify",
+                                rowtimeField,
+                                allFieldsTypes.keySet().stream()
+                                        .collect(Collectors.joining("`, `", "[`", "`]"))));
+            }
+        }
+
+        @Nullable
+        private String getComment(SqlTableColumn column) {
+            return column.getComment()
+                    .map(SqlCharStringLiteral.class::cast)
+                    .map(c -> c.getValueAs(String.class))
+                    .orElse(null);
+        }
+
+        private void collectColumnPosition(
+                AlterSchemaStrategy strategy, List<SqlNode> alterColumns) {
+            Set<String> alterColNames = new HashSet<>();
+            for (SqlNode alterColumn : alterColumns) {
+                SqlTableColumnPosition columnPosition = (SqlTableColumnPosition) alterColumn;
+                SqlTableColumn column = columnPosition.getColumn();
+                String name = column.getName().getSimple();
+                boolean duplicate = !alterColNames.add(name);
+                boolean existed = sortedColumnNames.contains(name);
+                boolean first = columnPosition.isFirstColumn();
+                boolean after = columnPosition.isAfterReferencedColumn();
+                if (duplicate) {
+                    String errorMsg =
+                            String.format(
+                                    "Try to %s duplicate column `%s`.",
+                                    strategy == AlterSchemaStrategy.ADD
+                                            ? "add"
+                                            : "modify table with",
+                                    name);

Review Comment:
   Spark's?



##########
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) {

Review Comment:
   unresolvedSchema -> originalSchema/oldSchema?



##########
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);
+            watermarkSpecs.clear();

Review Comment:
   Why clear? I think overwrite is okay?



##########
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);
+                }

Review Comment:
   why not 
   ```
   if (alterColumn instanceOf SqlComputedColumn) {
   
   } else if (alterColumn instanceOf SqlMetadataColumn) {
   
   } else if (...) {
   
   }
   ```
   
   Other people don't need to understand what is non-computed column.



##########
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? Why not throw exception to notify users?  It's better we don't assume it's concated the name with `.`.



##########
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);
+            watermarkSpecs.clear();
+            watermarkSpecs.put(
+                    rowtimeField,
+                    new Schema.UnresolvedWatermarkSpec(
+                            rowtimeField,
+                            new SqlCallExpression(
+                                    escapeExpressions.apply(
+                                            alterWatermarkSpec.getWatermarkStrategy()))));
+        }
+
+        private void convertNonComputedColumn(SqlTableColumn column) {
+            boolean isPhysical = column instanceof SqlTableColumn.SqlRegularColumn;
+            Schema.UnresolvedColumn newColumn =
+                    isPhysical
+                            ? convertPhysicalColumn((SqlTableColumn.SqlRegularColumn) column)
+                            : convertMetadataColumn((SqlTableColumn.SqlMetadataColumn) column);
+            columns.put(column.getName().getSimple(), newColumn);
+        }
+
+        Schema.UnresolvedPhysicalColumn convertPhysicalColumn(
+                SqlTableColumn.SqlRegularColumn physicalColumn) {
+            DataType dataType = getDataType(physicalColumn.getType());
+            return new Schema.UnresolvedPhysicalColumn(
+                    physicalColumn.getName().getSimple(), dataType, getComment(physicalColumn));
+        }
+
+        private Schema.UnresolvedMetadataColumn convertMetadataColumn(
+                SqlTableColumn.SqlMetadataColumn metadataColumn) {
+            DataType dataType = getDataType(metadataColumn.getType());
+            return new Schema.UnresolvedMetadataColumn(
+                    metadataColumn.getName().getSimple(),
+                    dataType,
+                    metadataColumn.getMetadataAlias().orElse(null),
+                    metadataColumn.isVirtual(),
+                    getComment(metadataColumn));
+        }
+
+        private void convertComputedColumn(SqlTableColumn.SqlComputedColumn column) {
+            String columnName = column.getName().getSimple();
+            Schema.UnresolvedColumn newColumn =
+                    new Schema.UnresolvedComputedColumn(
+                            columnName,
+                            new SqlCallExpression(escapeExpressions.apply(column.getExpr())),
+                            getComment(column));
+            columns.put(columnName, newColumn);
+        }
+
+        private DataType getDataType(SqlDataTypeSpec typeSpec) {
+            RelDataType relType =
+                    typeSpec.deriveType(
+                            sqlValidator, typeSpec.getNullable() == null || typeSpec.getNullable());
+            return fromLogicalToDataType(toLogicalType(relType));
+        }
+
+        @Nullable
+        String getComment(SqlTableColumn column) {
+            return column.getComment()
+                    .map(SqlCharStringLiteral.class::cast)
+                    .map(c -> c.getValueAs(String.class))
+                    .orElse(null);
+        }
+
+        private void collectColumnPosition(List<SqlNode> alterColumns) {
+            for (SqlNode alterColumn : alterColumns) {
+                SqlTableColumnPosition columnPosition = (SqlTableColumnPosition) alterColumn;
+                SqlTableColumn column = columnPosition.getColumn();
+                if (!column.getName().isSimple()) {
+                    throw new UnsupportedOperationException(
+                            String.format(
+                                    "%sAlter nested row type is not supported yet.",
+                                    EX_MSG_PREFIX));
+                }
+                String columnName = column.getName().getSimple();
+                boolean first = columnPosition.isFirstColumn();
+                boolean after = columnPosition.isAfterReferencedColumn();
+                if (!alterColNames.add(columnName)) {
+                    throw new ValidationException(
+                            String.format(
+                                    "%sEncounter duplicate column `%s`.",
+                                    EX_MSG_PREFIX, columnName));
+                }
+                checkColumnExists(columnName);
+                if (first) {
+                    addFirst(columnName);
+                } else if (after) {
+                    addAfter(columnName, getReferencedColumn(columnPosition));
+                } else {
+                    addLast(columnName);
+                }
+            }
+        }
+
+        private String getReferencedColumn(SqlTableColumnPosition columnPosition) {
+            SqlIdentifier referencedIdent = columnPosition.getAfterReferencedColumn();
+            Preconditions.checkNotNull(
+                    referencedIdent,
+                    String.format("%sCould not refer to a null column", EX_MSG_PREFIX));
+            if (!referencedIdent.isSimple()) {
+                throw new UnsupportedOperationException(
+                        String.format(
+                                "%sAlter nested row type is not supported yet.", EX_MSG_PREFIX));
+            }
+            String referencedName = referencedIdent.getSimple();
+            if (!sortedColumnNames.contains(referencedName)) {
+                throw new ValidationException(
+                        String.format(
+                                "%sReferenced column `%s` by 'AFTER' does not exist in the table.",
+                                EX_MSG_PREFIX, referencedName));
+            }
+            return referencedName;
+        }
+
+        private Schema convert() {
+            List<Schema.UnresolvedColumn> newColumns = new ArrayList<>();
+            for (String column : sortedColumnNames) {
+                newColumns.add(columns.get(column));
+            }
+            Schema.Builder resultBuilder = Schema.newBuilder().fromColumns(newColumns);
+            if (primaryKey != null) {
+                String constraintName = primaryKey.getConstraintName();
+                List<String> pkColumns = primaryKey.getColumnNames();
+                if (constraintName != null) {
+                    resultBuilder.primaryKeyNamed(constraintName, pkColumns);
+                } else {
+                    resultBuilder.primaryKey(pkColumns);
+                }
+            }
+            watermarkSpecs.forEach(
+                    (k, v) -> resultBuilder.watermark(k, v.getWatermarkExpression()));
+            Schema updatedSchema = resultBuilder.build();
+            try {
+                schemaResolver.resolve(updatedSchema);
+                return updatedSchema;
+            } catch (Exception e) {
+                throw new ValidationException(String.format("%s%s", EX_MSG_PREFIX, e.getMessage()));
+            }
+        }
+
+        abstract void checkColumnExists(String columnName);
+
+        abstract void checkPrimaryKeyExists();
+
+        abstract void checkWatermarkExists();
+
+        abstract void addFirst(String columnName);
+
+        abstract void addAfter(String columnName, String referencedColumn);
+
+        abstract void addLast(String columnName);
+    }
+
+    private static class AddSchemaConverter extends SchemaConverter {
+
+        AddSchemaConverter(
+                Schema unresolvedSchema,
+                FlinkTypeFactory typeFactory,
+                SqlValidator sqlValidator,
+                Function<SqlNode, String> escapeExpressions,
+                SchemaResolver schemaResolver) {
+            super(unresolvedSchema, typeFactory, sqlValidator, escapeExpressions, schemaResolver);
+        }
+
+        @Override
+        void checkPrimaryKeyExists() {
+            if (primaryKey != null) {
+                throw new ValidationException(
+                        String.format(
+                                "%sThe base table has already defined the primary key constraint %s. You might "
+                                        + "want to drop it before adding a new one.",
+                                EX_MSG_PREFIX,
+                                primaryKey.getColumnNames().stream()
+                                        .collect(Collectors.joining("`, `", "[`", "`]"))));
+            }
+        }
+
+        @Override
+        void checkWatermarkExists() {
+            if (!watermarkSpecs.isEmpty()) {

Review Comment:
   It's not very clear here. With the map defination, it's legal that every rowtime field has one watermark strategy. But the current implementation means we can have at most one watermark strategy here. I think we can releax the restriction here?
   
   BTW, I think it's fine to introduce a method named applyWatermarkChange here. It's much clear for others to understand the behaviour is different between MODIFY and ADD syntax.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org