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


##########
flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java:
##########
@@ -232,6 +232,118 @@ void testConsumingAllEvents(boolean parallelismSnapshot)
         result.getJobClient().get().cancel().get();
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    void testConsumingAllEventsForPartitionedTable(boolean parallelismSnapshot)
+            throws SQLException, ExecutionException, InterruptedException {
+        setup(parallelismSnapshot);
+        initializePostgresTable(POSTGRES_CONTAINER, "inventory_partitioned");
+        String publicationName = "dbz_publication_" + new 
Random().nextInt(1000);
+        String slotName = getSlotName();
+        try (Connection connection = getJdbcConnection(POSTGRES_CONTAINER);
+                Statement statement = connection.createStatement()) {
+            statement.execute(
+                    String.format(
+                            "CREATE PUBLICATION %s FOR TABLE 
inventory_partitioned.products "
+                                    + " WITH 
(publish_via_partition_root=true)",
+                            publicationName));
+            statement.execute(
+                    String.format(
+                            "select 
pg_create_logical_replication_slot('%s','pgoutput');",
+                            slotName));
+        }
+
+        String sourceDDL =
+                String.format(
+                        "CREATE TABLE debezium_source ("
+                                + " id INT NOT NULL,"
+                                + " name STRING,"
+                                + " description STRING,"
+                                + " weight DECIMAL(10,3),"
+                                + " country STRING"
+                                + ") WITH ("
+                                + " 'connector' = 'postgres-cdc',"
+                                + " 'hostname' = '%s',"
+                                + " 'port' = '%s',"
+                                + " 'username' = '%s',"
+                                + " 'password' = '%s',"
+                                + " 'database-name' = '%s',"
+                                + " 'schema-name' = '%s',"
+                                + " 'table-name' = '%s',"
+                                + " 'scan.incremental.snapshot.enabled' = 
'%s',"
+                                + " 'decoding.plugin.name' = 'pgoutput', "
+                                + " 'debezium.publication.name'  = '%s',"
+                                + " 'slot.name' = '%s'"
+                                + ")",
+                        POSTGRES_CONTAINER.getHost(),
+                        POSTGRES_CONTAINER.getMappedPort(POSTGRESQL_PORT),
+                        POSTGRES_CONTAINER.getUsername(),
+                        POSTGRES_CONTAINER.getPassword(),
+                        POSTGRES_CONTAINER.getDatabaseName(),
+                        "inventory_partitioned",
+                        "products",
+                        parallelismSnapshot,
+                        publicationName,
+                        slotName);
+        String sinkDDL =
+                "CREATE TABLE sink ("
+                        + " id INT NOT NULL,"
+                        + " name STRING,"
+                        + " description STRING,"
+                        + " weight DECIMAL(10,3),"
+                        + " country STRING,"
+                        + " PRIMARY KEY (id, country) NOT ENFORCED"
+                        + ") WITH ("
+                        + " 'connector' = 'values',"
+                        + " 'sink-insert-only' = 'false',"
+                        + " 'sink-expected-messages-num' = '20'"
+                        + ")";
+        tEnv.executeSql(sourceDDL);
+        tEnv.executeSql(sinkDDL);
+
+        // async submit job
+        TableResult result =
+                tEnv.executeSql(
+                        "INSERT INTO sink SELECT id, name, description, 
weight, country FROM debezium_source");
+
+        waitForSnapshotStarted("sink");
+
+        // wait a bit to make sure the replication slot is ready
+        Thread.sleep(5000);
+
+        // generate WAL
+        try (Connection connection = getJdbcConnection(POSTGRES_CONTAINER);
+                Statement statement = connection.createStatement()) {

Review Comment:
   We can also create a new sub-partition table and insert data here. Newly 
added partition table will also be read.



##########
flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/utils/TableDiscoveryUtils.java:
##########
@@ -38,7 +38,8 @@ public static List<TableId> listTables(
             throws SQLException {
 
         Set<TableId> allTableIds =
-                jdbc.readTableNames(database, null, null, new String[] 
{"TABLE"});

Review Comment:
   I hope we can add a param such as partitioned(maybe we can check whether 
debezium has), when this param is enabled:
   1. discovery table here.
   2. add publish_via_partition_root=true when create publication in 
io.debezium.connector.postgresql.connection.PostgresReplicationConnection#initPublication.
   
   To be honest, I hope debezium helps do it. But if it lacks it, we can do it 
to make easier for user.



-- 
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: issues-unsubscr...@flink.apache.org

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

Reply via email to