-
Notifications
You must be signed in to change notification settings - Fork 2k
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
[FLINK-37191][mysql] Avoid back filling if the lowWatermark of BinlogSplit is equal to highWatermark. #3902
Conversation
@yuxiqian @leonardBang PTAL. |
CI failed.... |
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.
BinlogOffset#compareTo
is a general-purposed comparison method, and this change might affect more than backfill necessity check. Is it possible to tweak SnapshotSplitReader#isBackfillRequired
only?
Seems |
Hi, @JNSimba. I found that the change of #3415 will cause the following problem:
I thank that the change of #3415 mainly to solve the problem of the comparson of highWatermark and the last record, but in reality, we don't need to read this record, so we can skip it directly. Line 102 in 7717779
Notice that this will not cause data loss, as this record will be read in backfill phase. What do you think? |
@lvyanquan Is the backfill phase not stopped because the Lines 99 to 102 in 7717779
|
… and high watermark " This reverts commit 2fa215e.
… to maxSplitHighWatermark.
1669909
to
5e57e72
Compare
Yes.
Make sense. So I would like to revert the change of adding timestamp to highWaterMark and skip the record that is equal to the offset of highWaterMark. The test case of BinlogSplitReaderTest will be remained to verify this. |
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.
LGTM
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.
Pulled and tested locally, I could confirm that PolarDBX test case could run without problem.
But seems BinlogSplitReaderTest#testBinlogOffsetCompareWithSnapshotAndBinlogPhase
(introduced in #3415) will always fail on this assertion:
assertTrue(sourceRecords.isEmpty());
Expecting empty but was: [SourceRecord{sourcePartition={server=mysql_binlog_source}, sourceOffset={transaction_id=null, ts_sec=1739012804, file=mysql-bin.000003, pos=15691, row=1, server_id=223344, event=2}} ConnectRecord{topic='mysql_binlog_source.customer_tesuwt.customers', kafkaPartition=null, key=Struct{id=999999}, keySchema=Schema{mysql_binlog_source.customer_tesuwt.customers.Key:STRUCT}, value=Struct{after=Struct{id=999999,name=user_22,address=Shanghai,phone_number=123567891234},source=Struct{version=1.9.8.Final,connector=mysql,name=mysql_binlog_source,ts_ms=1739012804000,db=customer_tesuwt,table=customers,server_id=223344,file=mysql-bin.000003,pos=15831,row=0,thread=5},op=c,ts_ms=1739012808026}, valueSchema=Schema{mysql_binlog_source.customer_tesuwt.customers.Envelope:STRUCT}, timestamp=null, headers=ConnectHeaders(headers=)}]
… to maxSplitHighWatermark.
@@ -232,7 +232,9 @@ public int compareTo(BinlogOffset that) { | |||
// compared ... | |||
long timestamp = this.getTimestampSec(); | |||
long targetTimestamp = that.getTimestampSec(); | |||
return Long.compare(timestamp, targetTimestamp); | |||
if (timestamp != 0 && targetTimestamp != 0) { | |||
return Long.compare(timestamp, targetTimestamp); |
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.
This change is necessary because that if the TimestampSec is 0, the assumption of These are from different servers, and their binlog coordinates are not related
is not correct, these two BinlogOffsets could come from the same server.
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.
Pulled, ran tests locally and looks good.
Considering BinlogOffset
is part of public API and has complex comparison logic, maybe we can add some tests later to cover it in case if we break something accidentally again in the future.
To fix timeout in test case of PolardbxSourceITCase.
Avoid extra backfill when there is no binlog event between lowWatermark and highWatermark.