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


##########
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?
   
   Yes. According to the Jira discussion, it should be an operation to shift 
position.
   I also felt`addFirst`/`addAfter`/`addLast` is strange for ALTER TABLE 
MODIFY. 
   What about `placeFirst`/`placeAfter`/`place`



##########
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?
   
   Yes. According to the Jira discussion, it should be an operation to shift 
position.
   I also felt`addFirst`/`addAfter`/`addLast` is strange for ALTER TABLE 
MODIFY. 
   What about `placeFirst`/`placeAfter`/`place`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to