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



##########
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:
       We can also through a helpful exception if the column is used for 
watermarks. For computed column, we cannot do much.

##########
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:
       We can also throw a helpful exception if the column is used for 
watermarks. For computed column, we cannot do much.




-- 
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:
us...@infra.apache.org


Reply via email to