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


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/AlterTableSchemaUtil.java:
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.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.SqlCallExpression;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.NlsString;
+
+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.Optional;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Util to convert {@link SqlAlterTableSchema} with source table to generate 
new {@link Schema}. */
+public class AlterTableSchemaUtil {
+
+    private final SqlValidator sqlValidator;
+    private final Consumer<SqlTableConstraint> validateTableConstraint;
+    private final Function<SqlNode, String> escapeExpression;
+
+    AlterTableSchemaUtil(
+            SqlValidator sqlValidator,
+            Function<SqlNode, String> escapeExpression,
+            Consumer<SqlTableConstraint> validateTableConstraint) {
+        this.sqlValidator = sqlValidator;
+        this.validateTableConstraint = validateTableConstraint;
+        this.escapeExpression = escapeExpression;
+    }
+
+    public Schema convertSchema(
+            SqlAlterTableSchema alterTableSchema, ResolvedCatalogTable 
originalTable) {
+        UnresolvedSchemaBuilder builder =
+                new UnresolvedSchemaBuilder(
+                        originalTable,
+                        (FlinkTypeFactory) sqlValidator.getTypeFactory(),
+                        sqlValidator,
+                        validateTableConstraint,
+                        escapeExpression);
+        AlterSchemaStrategy strategy =
+                alterTableSchema instanceof SqlAlterTableAdd
+                        ? AlterSchemaStrategy.ADD
+                        : AlterSchemaStrategy.MODIFY;
+        builder.addOrModifyColumns(strategy, 
alterTableSchema.getColumns().getList());
+        List<SqlTableConstraint> fullConstraint = 
alterTableSchema.getFullConstraint();
+        if (!fullConstraint.isEmpty()) {
+            builder.addOrModifyPrimaryKey(strategy, fullConstraint.get(0));
+        }
+        alterTableSchema
+                .getWatermark()
+                .ifPresent(sqlWatermark -> 
builder.addOrModifyWatermarks(strategy, sqlWatermark));
+        return builder.build();
+    }
+
+    private static class UnresolvedSchemaBuilder {
+
+        List<String> newColumnNames = new ArrayList<>();
+        Set<String> alterColumnNames = new HashSet<>();
+        Map<String, Schema.UnresolvedColumn> columns = new HashMap<>();
+        Map<String, Schema.UnresolvedWatermarkSpec> watermarkSpec = new 
HashMap<>();

Review Comment:
   Correct me if I'm wrong, but I don't think we can support multiple 
watermarks.



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