-
Notifications
You must be signed in to change notification settings - Fork 587
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
refactor(cdc source): merge cdc heartbeat chunk builder & data chunk builder #19671
Merged
+85
−49
Merged
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
bedbc9b
merge cdc heartbeat chunk & data chunk
stdrc 5a0164f
check max txn size at the end of batch
stdrc d6e31cf
minor
stdrc 68f49f7
only emit heartbeat msg when there's no other msg
stdrc 3244845
improve readibility
stdrc 185defc
rename `take` to `take_and_reserve`
stdrc File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
The payload of postgres heartbeat message is not none after #19385. If you rely on the assumption that
payload.is_none()
, you should revisit your code logic.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.
The original logic on line 302 also relies on this, and it is the only place that appends to
heartbeat_builder
instead ofbuilder
. Do you mean that #19385 may cause the original logic to misplace heartbeat msgs intobuilder
as well?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.
For postgres source, the heartbeat message would not goes into the
heartbeat_builder
, since every time the heartbeat message will contain the payloadnow()::varchar
as show in #19385, which can be seen as a normal data chunk.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.
What if this
now()::varchar
row is at the end of a data chunk? Seems in this case thesnapshot_done
flag won't be update correctly, just like what we saw in the CI failure.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.
And, who will handle the
now()::varchar
row?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.
For postgres source it just same as other data chunk, the schema of cdc source is
payload jsonb
. I think you don't need to aware these details.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.
After some diving, now I understand this process. Let me try to summarize.
Why we need Debezium heartbeat?
As explained by Debezium document (quoted above), when Debezium is capturing a low-traffic database A, and at the same time there's another high-traffic database B in the PG instance, Debezium cannot "confirm the LSN", because the low-traffic database may have no traffic and hence won't trigger Debezium. To resolve this problem, we have to tell Debezium to "make some changes in database A periodically" so that the "LSN confirmation" can be triggered. This "periodic" behavior is controlled by
heartbeat.interval.ms
.How is a heartbeat message like?
After we set
heartbeat.interval.ms
, RisingWave will receive heartbeat message from Debezium periodically, which areSourceMessage
s like this:The
key
andpayload
are bothNone
, no matter whatheartbeat.action.query
is (we will come to this in the next section). When seeing a heartbeat message,connector::parser::into_chunk_stream_inner
should yield a chunk containing an invisible row with theoffset
in the message, so thatSourceExecutor
can then doupdate_in_place
and henceupdate_offset
for theDebeziumCdcSplit
metadata.However, like the document said, the LSN (inside
sourceOffset
field) will not change because the "LSN confirmation" process cannot be triggered.How does Debezium "make some changes"?
So now we need Debezium to "make some changes".
As the same document said, the operation used to generate changes is configured via
heartbeat.action.query
. An example for this config given in the document isINSERT INTO test_heartbeat_table (text) VALUES ('test_heartbeat')
, which just inserts a meaningless row to a pre-created meaningless "heartbeat table".In our case, we don't make changes by inserting into some "heartbeat table", we use
pg_logical_emit_message
, which "emits a logical decoding message" to "pass generic messages to logical decoding plugins through WAL". This PG function is also mentioned in Debezium document. When this function is called inside database A, a "message event" will be captured by Debezium, then it can do "LSN confirmation".With this config set to
SELECT pg_logical_emit_message(false, 'heartbeat', now()::varchar)
, we receive the following twoSourceMessage
s periodically:Note that, we will receive both the empty-payloaded heartbeat message (with
isHeartbeat: true
), and the "message event" generated bypg_logical_emit_message
belonging to themessage
table. And since Debezium captured a new event (the "message event"), it "confirms LSN", so theLSN
got updated.In our current implementation, the first message will be handled as heartbeat message and emitted as an invisible empty row with "offset", the second will be parsed as a regular row, like:
The
MjAyNC0xMi0wNiAyMzo0MjowMC44MDAwMDErMDg=
is the base64-encodednow()::varchar
, and theprefix
isheartbeat
which we set when callingpg_logical_emit_message
. And because thefull_table_name
ismessage
which does not match any table we want to replicate, this row will be dismissed later (I didn't dive in this part, so sorrect me if I'm wrong.)Both the
offset
of heartbeat message and of the second message will take effect when it comes toDebeziumCdcSplit::update_offset
.Now we can conclude that, the value of
now()::varchar
simply does not matter at all, and we can actually change it to an empty string or any other string to avoid confusion. And, the PR #19385, changing the first argumenttransactional
fromfalse
totrue
, will not significantly change theSourceMessage
s we mentioned above, instead, it just generates additional transactionBEGIN
andCOMMIT
messages before and after the secondSourceMessage
.On PR #19385
For the issue which #19385 wanted to resolve (#16697), I guess maybe a better fix will be to set the
flush
argument totrue
when callingpg_logical_emit_message
, instead of settingtransactional
to true.The PG document says:
From my understanding, what we actually want is to immediately flush the emitted message to WAL so that Debezium can capture it.