-
Bug
-
Resolution: Done
-
Blocker
-
0.9.0.Beta1
I have noticed that some records are present in CDC table but are not present in Kafka topic.
I found in my logs:
[2019-01-14 09:23:33,173] INFO Last LSN recorded in offsets is 00040266:000188e4:0083 (io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource:78) ... [2019-01-14 09:23:33,185] INFO Skipping change ChangeTablePointer [changeTable=ChangeTable [captureInstance=XXX, sourceTableId=YYY, changeTableId=ZZZ, startLsn=0003f43e:0001b337:0003, changeTableObjectId=1673773020, stopLsn=NULL], resultSet=SQLServerResultSet:92773414, completed=false, currentChangeLsn=00040266:0001866b:00b0] as its LSN is smaller than the last recorded LSN 00040266:000188e4:0083 (io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource:148) [2019-01-14 09:23:33,186] INFO Skipping change ChangeTablePointer [changeTable=ChangeTable [captureInstance=XXX, sourceTableId=YYY, changeTableId=ZZZ, startLsn=0003f43e:0001b337:0003, changeTableObjectId=1673773020, stopLsn=NULL], resultSet=SQLServerResultSet:92773414, completed=false, currentChangeLsn=00040266:0001866b:00b6] as its LSN is smaller than the last recorded LSN 00040266:000188e4:0083 (io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource:148)}} ...
I queried the capture instance with lsn range where I noticed data loss:
select [__$start_lsn], [__$seqval] from cdc.fn_cdc_get_all_changes_XXX(0x000402650001ab1c003a, 0x0004026700006dc600bf, 'all');
__$start_lsn (commitLsn) __$seqval (changeLsn) ... 00040266:0001852e:0003 00040266:000184e2:007d 00040266:0001856c:0003 00040266:00018559:0003 00040266:0001856c:0003 00040266:00018559:0009 00040266:00018a21:000d 00040266:000188e4:0081 00040266:00018a21:000d 00040266:000188e4:0083 <-- last recorded LSN before KConnect restart 00040266:00018fb8:0001 00040266:00018ae2:0120 00040266:00019015:0006 00040266:0001866b:00b0 00040266:00019015:0006 00040266:0001866b:00b6 00040266:00019015:0006 00040266:0001866b:00b8 00040266:00019015:0006 00040266:0001866b:00be ...
And these are records from my Kafka topic:
__$start_lsn (commitLsn) __$seqval (changeLsn) ... 00040266:0001852e:0003 00040266:000184e2:007d 00040266:0001856c:0003 00040266:00018559:0003 00040266:0001856c:0003 00040266:00018559:0009 00040266:00018a21:000d 00040266:000188e4:0081 00040266:00018a21:000d 00040266:000188e4:0083 <-- last fetched before KConnect restart -------------------------------------------------------- KConnect restart 00040266:00018fb8:0001 00040266:00018ae2:0120 <-- first fetched after restart KConnect restart 00040266:00019015:0006 00040266:00018ae2:00c3 00040266:00019015:0006 00040266:00018ae2:00c9 00040266:00019015:0006 00040266:00018ae2:00cb 00040266:00019015:0006 00040266:00018ae2:00d0 00040266:00019015:0006 00040266:00018ae2:00d2 00040266:00019015:0006 00040266:00018ae2:00d9 ...
As you can see from SQL Server query results commitLsn is monotonically increasing, while changeLsn is not (only within records with the same commitLsn)!
Please note:
- 00040266:00018a21:000d 00040266:000188e4:0083 <- last recorded LSN before KConnect restart
- 00040266:00019015:0006 00040266:00018ae2:0120 <- first fetched record after KConnect restart
- 00040266:00019015:0006 00040266:0001866b:00b0 <- should have been fetched but 00040266:0001866b:00b0 is lower than 00040266:000188e4:0083. Row has been skipped (see logs).
- 00040266:00019015:0006 00040266:0001866b:00b6 <- should have been fetched but 00040266:0001866b:00b6 is lower than 00040266:000188e4:0083. Row has been skipped (see logs).
- The only records fetched for commitLsn 00040266:00019015:0006 are records with changeLsn greater than 00040266:000188e4:0083.
changeLsn is not enough to reliably track offset in capture instances. According to docs:
__$seqval binary(10) Sequence value used to order changes to a row within a transaction.