loserwang1024 commented on code in PR #4316:
URL: https://github.com/apache/flink-cdc/pull/4316#discussion_r2946177770


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/utils/SchemaChangeUtil.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.cdc.connectors.postgres.utils;
+
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.RenameColumnEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.connectors.postgres.source.PostgresDialect;
+import 
org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceConfig;
+
+import io.debezium.connector.postgresql.PostgresConnectorConfig;
+import io.debezium.connector.postgresql.TypeRegistry;
+import io.debezium.connector.postgresql.connection.PostgresConnection;
+import io.debezium.relational.Column;
+import io.debezium.relational.Table;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.cdc.connectors.postgres.utils.PostgresSchemaUtils.toCdcTableId;
+import static 
org.apache.flink.cdc.connectors.postgres.utils.PostgresSchemaUtils.toSchema;
+
+/**
+ * Utilities for inferring CDC schema change events by comparing before/after 
Debezium table
+ * schemas.
+ *
+ * <p>Ideally, PostgreSQL's {@code pg_attribute.attnum} would be the best way 
to distinguish whether
+ * a column was dropped-and-recreated or simply renamed. However, the pgoutput 
relation message does
+ * not include {@code attnum}. Even querying the database for the current 
attnum after the fact is
+ * unreliable due to temporal mismatch — for example, if column {@code C} is 
renamed to {@code D}
+ * and a new column {@code C} is added between two relation messages, a 
retroactive attnum query
+ * would see the <em>new</em> {@code C}'s attnum, not the original one, 
leading to incorrect
+ * inference.
+ *
+ * <p>Instead, this utility infers schema changes by comparing the 
before/after column lists and
+ * computing the <b>minimum edit</b> using four operations: rename, drop, 
add-at-last, and
+ * alter-type. When multiple interpretations are possible (e.g. {@code 
[c1,c2,c3,c4,c5] →
+ * [c1,c2,c4,c5]} could be "drop c3" or "drop c5 then rename c3→c6, c4→..." 
etc.), the algorithm
+ * favors the one with the fewest total changes.
+ */
+public class SchemaChangeUtil {
+    /**
+     * Infers the minimum list of schema change events that transform {@code 
tableBefore} into
+     * {@code tableAfter}. Returns a {@link CreateTableEvent} if {@code 
tableBefore} is null.
+     */
+    public static List<SchemaChangeEvent> inferSchemaChangeEvent(
+            io.debezium.relational.TableId dbzTableId,
+            @Nullable Table tableBefore,
+            Table tableAfter,
+            PostgresSourceConfig sourceConfig,
+            PostgresDialect dialect) {
+
+        if (tableBefore == null) {
+            return Collections.singletonList(toCreateTableEvent(tableAfter, 
sourceConfig, dialect));
+        }
+
+        TableId cdcTableId =
+                toCdcTableId(
+                        dbzTableId,
+                        sourceConfig.getDatabaseList().get(0),
+                        sourceConfig.isIncludeDatabaseInTableId());
+        PostgresConnectorConfig dbzConfig = 
sourceConfig.getDbzConnectorConfig();
+
+        try (PostgresConnection connection = dialect.openJdbcConnection()) {
+            TypeRegistry typeRegistry = connection.getTypeRegistry();
+            return inferMinimalSchemaChanges(
+                    cdcTableId,
+                    tableBefore.columns(),
+                    tableAfter.columns(),
+                    dbzConfig,
+                    typeRegistry);
+        }
+    }
+
+    public static CreateTableEvent toCreateTableEvent(
+            Table table, PostgresSourceConfig sourceConfig, PostgresDialect 
dialect) {
+        try (PostgresConnection connection = dialect.openJdbcConnection()) {
+            return toCreateTableEvent(table, sourceConfig, 
connection.getTypeRegistry());
+        }
+    }
+
+    private static CreateTableEvent toCreateTableEvent(
+            Table table, PostgresSourceConfig sourceConfig, TypeRegistry 
typeRegistry) {
+        return new CreateTableEvent(
+                toCdcTableId(
+                        table.id(),
+                        sourceConfig.getDatabaseList().get(0),
+                        sourceConfig.isIncludeDatabaseInTableId()),
+                toSchema(table, sourceConfig.getDbzConnectorConfig(), 
typeRegistry));
+    }
+
+    /**
+     * Finds the minimum schema change events to transform beforeCols into 
afterCols using recursion
+     * with memoization. Available operations: rename column, add column at 
last, drop column, alter
+     * column type. Recursion depth bounded by total column count.
+     */
+    private static List<SchemaChangeEvent> inferMinimalSchemaChanges(

Review Comment:
   @Daishuyuan As far as I understand, PostgreSQL does not support inserting a 
column at the head of a table. ALTER TABLE ... ADD COLUMN only appends the new 
column to the end of the column list:
   
   https://www.postgresql.org/docs/current/sql-altertable.html
   
   So in a case like:
   
   before: [b, c]
   after: [a, b, c]
   a true “head-insert” is not something PostgreSQL would normally produce 
through native schema evolution on the same table.
   
   The reason I used this algorithm is that I am trying, as much as possible, 
to preserve the original DDL semantics that the source database can actually 
produce. I’ll add this explanation near the beginning of the method to make the 
assumption clearer.
   
   In the current design, the main ambiguous case is the last-column scenario: 
for example, distinguishing rename of the last column from drop + add of a last 
column with the same name. Aside from that, in most cases the inferred result 
is aligned with what the database itself can express.
   
   So yes, this behavior is intentional under the current design constraints, 
rather than an attempt to find the most “natural” edit sequence in the abstract



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/utils/SchemaChangeUtil.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.cdc.connectors.postgres.utils;
+
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.RenameColumnEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.connectors.postgres.source.PostgresDialect;
+import 
org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceConfig;
+
+import io.debezium.connector.postgresql.PostgresConnectorConfig;
+import io.debezium.connector.postgresql.TypeRegistry;
+import io.debezium.connector.postgresql.connection.PostgresConnection;
+import io.debezium.relational.Column;
+import io.debezium.relational.Table;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.cdc.connectors.postgres.utils.PostgresSchemaUtils.toCdcTableId;
+import static 
org.apache.flink.cdc.connectors.postgres.utils.PostgresSchemaUtils.toSchema;
+
+/**
+ * Utilities for inferring CDC schema change events by comparing before/after 
Debezium table
+ * schemas.
+ *
+ * <p>Ideally, PostgreSQL's {@code pg_attribute.attnum} would be the best way 
to distinguish whether
+ * a column was dropped-and-recreated or simply renamed. However, the pgoutput 
relation message does
+ * not include {@code attnum}. Even querying the database for the current 
attnum after the fact is
+ * unreliable due to temporal mismatch — for example, if column {@code C} is 
renamed to {@code D}
+ * and a new column {@code C} is added between two relation messages, a 
retroactive attnum query
+ * would see the <em>new</em> {@code C}'s attnum, not the original one, 
leading to incorrect
+ * inference.
+ *
+ * <p>Instead, this utility infers schema changes by comparing the 
before/after column lists and
+ * computing the <b>minimum edit</b> using four operations: rename, drop, 
add-at-last, and
+ * alter-type. When multiple interpretations are possible (e.g. {@code 
[c1,c2,c3,c4,c5] →
+ * [c1,c2,c4,c5]} could be "drop c3" or "drop c5 then rename c3→c6, c4→..." 
etc.), the algorithm
+ * favors the one with the fewest total changes.
+ */
+public class SchemaChangeUtil {
+    /**
+     * Infers the minimum list of schema change events that transform {@code 
tableBefore} into
+     * {@code tableAfter}. Returns a {@link CreateTableEvent} if {@code 
tableBefore} is null.
+     */
+    public static List<SchemaChangeEvent> inferSchemaChangeEvent(
+            io.debezium.relational.TableId dbzTableId,
+            @Nullable Table tableBefore,
+            Table tableAfter,
+            PostgresSourceConfig sourceConfig,
+            PostgresDialect dialect) {
+
+        if (tableBefore == null) {
+            return Collections.singletonList(toCreateTableEvent(tableAfter, 
sourceConfig, dialect));
+        }
+
+        TableId cdcTableId =
+                toCdcTableId(
+                        dbzTableId,
+                        sourceConfig.getDatabaseList().get(0),
+                        sourceConfig.isIncludeDatabaseInTableId());
+        PostgresConnectorConfig dbzConfig = 
sourceConfig.getDbzConnectorConfig();
+
+        try (PostgresConnection connection = dialect.openJdbcConnection()) {
+            TypeRegistry typeRegistry = connection.getTypeRegistry();
+            return inferMinimalSchemaChanges(
+                    cdcTableId,
+                    tableBefore.columns(),
+                    tableAfter.columns(),
+                    dbzConfig,
+                    typeRegistry);
+        }
+    }
+
+    public static CreateTableEvent toCreateTableEvent(
+            Table table, PostgresSourceConfig sourceConfig, PostgresDialect 
dialect) {
+        try (PostgresConnection connection = dialect.openJdbcConnection()) {
+            return toCreateTableEvent(table, sourceConfig, 
connection.getTypeRegistry());
+        }
+    }
+
+    private static CreateTableEvent toCreateTableEvent(
+            Table table, PostgresSourceConfig sourceConfig, TypeRegistry 
typeRegistry) {
+        return new CreateTableEvent(
+                toCdcTableId(
+                        table.id(),
+                        sourceConfig.getDatabaseList().get(0),
+                        sourceConfig.isIncludeDatabaseInTableId()),
+                toSchema(table, sourceConfig.getDbzConnectorConfig(), 
typeRegistry));
+    }
+
+    /**
+     * Finds the minimum schema change events to transform beforeCols into 
afterCols using recursion
+     * with memoization. Available operations: rename column, add column at 
last, drop column, alter
+     * column type. Recursion depth bounded by total column count.
+     */
+    private static List<SchemaChangeEvent> inferMinimalSchemaChanges(

Review Comment:
   @Daishuyuan 
   As far as I understand, PostgreSQL does not support inserting a column at 
the head of a table. ALTER TABLE ... ADD COLUMN only appends the new column to 
the end of the column list:
   
   https://www.postgresql.org/docs/current/sql-altertable.html
   
   So in a case like:
   
   before: [b, c]
   after: [a, b, c]
   a true “head-insert” is not something PostgreSQL would normally produce 
through native schema evolution on the same table.
   
   The reason I used this algorithm is that I am trying, as much as possible, 
to preserve the original DDL semantics that the source database can actually 
produce. I’ll add this explanation near the beginning of the method to make the 
assumption clearer.
   
   In the current design, the main ambiguous case is the last-column scenario: 
for example, distinguishing rename of the last column from drop + add of a last 
column with the same name. Aside from that, in most cases the inferred result 
is aligned with what the database itself can express.
   
   So yes, this behavior is intentional under the current design constraints, 
rather than an attempt to find the most “natural” edit sequence in the abstract



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