[FLINK-37479][postgres] Add support for PARTITIONED TABLE
Add support for Postgres partitioned table 2 test cases added:
- Discover partitioned table
- IT test to verify snapshot and wal events can be consumed for partitioned table
Hello @loserwang1024 , could you help to review when you have time. Thank you
I have dedug this modification, it turns out.
jdbc.readTableNames(database, null, null, new String[] {"TABLE"}) will show sub partition tables:
jdbc.readTableNames( database, null, null, new String[] {"PARTITIONED TABLE"}) will show parents partition table:
Thus, if set table-name = 'inventory_partitioned.*', both leaf tables and parents table are captured.
Thus, even though we can not publish_via_partition_root in this pr, a option to enable partition table is needed:
Set<TableId> allTableIds ;
if(!partition) {
jdbc.readTableNames(
database, null, null, new String[]{ "TABLE"});
}else{
jdbc.readTableNames(
database, null, null, new String[]{ "PARTITIONED TABLE"});
}
Otherwise, this pr will influence currently regex logic(data duplicate)
hi @loserwang1024 , I've updated the PR to add this feature via a boolean flag (default to false) so it won't affect existing users. I also added a note on the docs that users who opt-in to use this feature should create PUBLICATION beforehand and avoid listing parent/child tables twice. Please let me know what you think.
I agree with you in general. Maybe only a minor discuss on option name. What about scan.publish-via-partition-root.enabled? @leonardBang @ruanhang1993 , WDYT?
<td>include-partitioned-table</td>
<td>optional</td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>
Whether to enable reading partitioned table.<br>
If enabled:
(1) PUBLICATION must be created beforehand with parameter publish_via_partition_root=true
(2) Table list (regex or predefined list) should only match the parent table name, if table list matches both parent and child tables, snapshot data will be read twice.
</td>
</tr>
hi @loserwang1024 , i've updated the option name based on your suggestion
I agree with you in general. Maybe only a minor discuss on option name. What about
scan.publish-via-partition-root.enabled? @leonardBang @ruanhang1993 , WDYT?<td>include-partitioned-table</td> <td>optional</td> <td style="word-wrap: break-word;">false</td> <td>Boolean</td> <td> Whether to enable reading partitioned table.<br> If enabled: (1) PUBLICATION must be created beforehand with parameter publish_via_partition_root=true (2) Table list (regex or predefined list) should only match the parent table name, if table list matches both parent and child tables, snapshot data will be read twice. </td> </tr>
Hi, @phamvinh1712 @loserwang1024 .Sorry for my late response.
IMO, I think this new option name scan.publish-via-partition-root.enabled is difficult to understand for users who are not familiar with postgres. Actually I am one of them. ;)
The first time I saw this option name, I was confused and do not know when I should open this option.
I think the option name could be like scan.include-partitioned-tables.enabled. WDYT?
scan.include-partitioned-tables.enabled.
+1 for this option name, I think it's better to not expose to many underlying concept to users, especially many Flink CDC users are not familiar with PG internal.
I agree with you guys. It seems that I was considered from a postgres connector developer rather than a norman user.
hi @loserwang1024 @leonardBang @ruanhang1993 , thanks all for the feedbacks 🙌 and sorry for the late response. I've updated the configuration like suggested, could you help to review the PR again.
CC @Mrart, as you may be interested in this.
CC @Mrart, as you may be interested in this.
Yes, thanks. @phamvinh1712 Then can we solve the UT test first? I will also take a close look at this pr.
hi @Mrart , thanks for reviewing. I merge master to my branch and the test is fixed now. Could any committer help to review/approve the PR to trigger UT tests again, thanks in advance