-
Notifications
You must be signed in to change notification settings - Fork 2.1k
[FLINK-39265][postgres] PostgreSQL CDC intermittently drops INSERT records after checkpoint recovery due to WalPositionLocator filtering #4323
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
|
|
||
| import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; | ||
|
|
||
| import io.debezium.connector.postgresql.PostgresOffsetContext; | ||
| import io.debezium.connector.postgresql.SourceInfo; | ||
| import io.debezium.connector.postgresql.connection.Lsn; | ||
| import io.debezium.time.Conversions; | ||
|
|
@@ -43,7 +44,34 @@ public class PostgresOffset extends Offset { | |
|
|
||
| // used by PostgresOffsetFactory | ||
| PostgresOffset(Map<String, String> offset) { | ||
| this.offset = offset; | ||
| Map<String, String> filtered = new HashMap<>(offset); | ||
| // When a checkpoint is taken right after a COMMIT (state-3a), all three LSN fields | ||
| // converge to the same value: lsn == lsn_proc == lsn_commit. | ||
| // Recovering from such a checkpoint constructs WalPositionLocator(C0, C0), which | ||
| // causes the first new transaction's DML records (data_start=C0 in pgoutput) to be | ||
| // silently dropped: they are added to lsnSeen during the find phase, but | ||
| // startStreamingLsn is set to the next COMMIT (C1), so they are filtered in the | ||
| // stream phase. | ||
| // | ||
| // Fix: when lsn == lsn_proc == lsn_commit, remove lsn_proc and lsn_commit so that | ||
| // WalPositionLocator is constructed with lastCommitStoredLsn=null, which triggers | ||
| // the fast path: startStreamingLsn=firstLsnReceived=C0, all messages pass through. | ||
| // | ||
| // The triple-equality condition is safe: mid-transaction checkpoints (state-3b) have | ||
| // lsn_commit pointing to the previous commit, so lsn_commit != lsn, and this branch | ||
| // is not taken. | ||
| // | ||
| // This workaround can be removed once Debezium is upgraded to a version that | ||
| // includes DBZ-6204: | ||
| // https://github.com/debezium/debezium/commit/3b5740f1a836c8b438888f2458ebb1554320bac7 | ||
| String lsnVal = filtered.get(SourceInfo.LSN_KEY); | ||
| String lsnProc = filtered.get(PostgresOffsetContext.LAST_COMPLETELY_PROCESSED_LSN_KEY); | ||
| String lsnCommit = filtered.get(PostgresOffsetContext.LAST_COMMIT_LSN_KEY); | ||
| if (lsnVal != null && lsnVal.equals(lsnProc) && lsnVal.equals(lsnCommit)) { | ||
| filtered.remove(PostgresOffsetContext.LAST_COMPLETELY_PROCESSED_LSN_KEY); | ||
| filtered.remove(PostgresOffsetContext.LAST_COMMIT_LSN_KEY); | ||
| } | ||
|
Comment on lines
+56
to
+73
|
||
| this.offset = filtered; | ||
|
Comment on lines
+47
to
+74
|
||
| } | ||
|
|
||
| PostgresOffset(Long lsn, Long txId, Instant lastCommitTs) { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
PostgresOffset.of(...)already builds a newHashMap(offsetStrMap) and then this constructor creates anotherHashMapcopy (filtered). SincegetStreamOffset()is called per streamed record (seeIncrementalSourceStreamFetcher.shouldEmit), this adds an extra allocation per event. Consider avoiding the second copy in the common case (e.g., only copy when the triple-equality condition is met, otherwise keep the passed-in map), or perform the filtering before constructing thePostgresOffsetwhen the input map is known to be newly created.