flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] todd5167 commented on a change in pull request #15755: [FLINK-22318][table] Support RENAME column name for ALTER TABLE state…
Date Thu, 29 Apr 2021 11:59:37 GMT

todd5167 commented on a change in pull request #15755:
URL: https://github.com/apache/flink/pull/15755#discussion_r622333275



##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/OperationConverterUtils.java
##########
@@ -144,6 +149,116 @@ public static Operation convertChangeColumn(
         // TODO: handle watermark and constraints
     }
 
+    public static Operation convertRenameColumn(
+            ObjectIdentifier tableIdentifier,
+            String originColumnName,
+            String newColumnName,
+            CatalogTable catalogTable) {
+
+        Schema modifiedTableSchema = catalogTable.getUnresolvedSchema();
+        validateColumnName(originColumnName, newColumnName, modifiedTableSchema);
+
+        Schema.Builder builder = Schema.newBuilder();
+        // build column
+        modifiedTableSchema.getColumns().stream()
+                .forEach(
+                        column -> {
+                            if (StringUtils.equals(column.getName(), originColumnName)) {
+                                buildNewColumnFromOriginColumn(builder, column, newColumnName);
+                            } else {
+                                buildNewColumnFromOriginColumn(builder, column, column.getName());
+                            }
+                        });
+        // build primary key column
+        List<String> originPrimaryKeyNames =
+                modifiedTableSchema
+                        .getPrimaryKey()
+                        .map(Schema.UnresolvedPrimaryKey::getColumnNames)
+                        .orElseGet(Lists::newArrayList);
+
+        List<String> newPrimaryKeyNames =
+                originPrimaryKeyNames.stream()
+                        .map(
+                                pkName ->
+                                        StringUtils.equals(pkName, originColumnName)
+                                                ? newColumnName
+                                                : pkName)
+                        .collect(Collectors.toList());
+
+        if (newPrimaryKeyNames.size() > 0) {
+            builder.primaryKey(newPrimaryKeyNames);
+        }
+        // build watermark
+        modifiedTableSchema.getWatermarkSpecs().stream()
+                .forEach(
+                        watermarkSpec -> {
+                            String watermarkRefColumnName = watermarkSpec.getColumnName();
+                            Expression watermarkExpression = watermarkSpec.getWatermarkExpression();
+                            if (StringUtils.equals(watermarkRefColumnName, originColumnName))
{
+                                String newWatermarkExpression =
+                                        ((SqlCallExpression) watermarkExpression)
+                                                .getSqlExpression()
+                                                .replace(watermarkRefColumnName, newColumnName);

Review comment:
       hi, @wuchong.
      If the calculated column expression contains a renamed column, it will throw 'org.apache.calcite.sql.validate.SqlValidatorException:
Unknown identifier 'oldColumn'
    ' when resolving the schema.
   
     You said 'traverse the node tree to check whether it contains the renamed column', I
don’t know where to add it. If use ResolverRule, how to distinguish it from ResolveSqlCallRule.
   
   
      thanks .




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

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



Mime
View raw message