Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-5464

Snapshot result not saved if LAST record is filtered out

    XMLWordPrintable

Details

    Description

      We have filtering enabled to discard certain data events.

       

      We faced problem that  snapshot completed successfully, but the result was not saved, and after restart the snapshot was seen as cancelled, and hence new snapshot attempt was made.

      Log explains sequence of actions:

      ... snapshot completed 
      
      2022-07-31 20:13:28,460 INFO [reporting-connector-jul29|task-0] Snapshot - Final stage (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource) [debezium-sqlserverconnector-reporting-connector-jul29-change-event-source-coordinator]
      
      2022-07-31 20:13:28,466 INFO [reporting-connector-jul29|task-0] Snapshot ended with SnapshotResult [status=COMPLETED, offset=SqlServerOffsetContext [sourceInfoSchema=Schema{io.debezium.connector.sqlserver.Source:STRUCT}, sourceInfo=SourceInfo [serverName=reporting-connector-jul29, changeLsn=NULL, commitLsn=000add39:000269d0:0032, eventSerialNo=null, snapshot=FALSE, sourceTime=2022-07-31T20:13:28.456Z], snapshotCompleted=true, eventSerialNo=1]] (io.debezium.pipeline.ChangeEventSourceCoordinator) [debezium-sqlserverconnector-reporting-connector-jul29-change-event-source-coordinator]
       
      ... but then when connector is restarted
      
      2022-07-31 20:25:32,823 INFO [reporting-connector-jul29|task-0] Found previous partition offset SqlServerPartition [sourcePartition={server=reporting-connector-jul29}]: {commit_lsn=000add39:000269d0:0032, snapshot=true, snapshot_completed=false} (io.debezium.connector.common.BaseSourceTask) [task-thread-reporting-connector-jul29-0]
      
      2022-07-31 20:25:35,989 INFO [reporting-connector-jul29|task-0] Snapshot step 1 - Preparing (io.debezium.relational.RelationalSnapshotChangeEventSource) [debezium-sqlserverconnector-reporting-connector-jul29-change-event-source-coordinator]
      
      2022-07-31 20:25:35,989 INFO [reporting-connector-jul29|task-0] Previous snapshot was cancelled before completion; a new snapshot will be taken. (io.debezium.relational.RelationalSnapshotChangeEventSource) [debezium-sqlserverconnector-reporting-connector-jul29-change-event-source-coordinator]
      

      This problem happened when the LAST record was filtered out, and the offset attached to it - being successful snapshot result: {"commit_lsn":"000add39:000269d0:0032","snapshot":true,"snapshot_completed":true} - was never saved.

       

      As a suggestion of the solution, the offsets describing snapshot result are more of technical nature, and should not be conveyed using standard data change events (as connector is unaware of their future processing in kafka-connect), but should rather be bound to technical events, e.g. history events.

       

      As a workaround we're planning to add extra/dummy table to be snapshotted as the last one, but without filtering. That way LAST event will be published, and the offset will be saved properly.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              kgrzechnik Krzysztof Grzechnik
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: