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

Blocking ad-hoc snapshot is not really blocking for MySQL

XMLWordPrintable

    • Icon: Bug Bug
    • Resolution: Done
    • Icon: Critical Critical
    • 2.5.0.Alpha1
    • 2.4.0.Final
    • mysql-connector
    • None
    • Important

      In order to make your issue reports as actionable as possible, please provide the following information, depending on the issue type.

      Bug report

      For bug reports, provide this information, please:

      What Debezium connector do you use and what version?

      2.4.0.final

      What is the connector configuration?

      Doesn't matter.

      What is the captured database version and mode of depoyment?

      Any MySQL version supported by Debezium.

      What behaviour do you expect?

      When the blocking ad-hoc snapshot is triggered streaming actually stops and only resumes when the snapshot is finished.

      What behaviour do you see?

      Streaming continues which creates multiple issues downstream.

      Do you see the same behaviour using the latest relesead Debezium version?

      Yes. Same behavior since introduction of the blocking ad-hoc snapshots. 

      Do you have the connector logs, ideally from start till finish?

      Log is attached

      How to reproduce the issue using our tutorial deployment?

      1. There must be multiple relatively large (~50K records each) monitored tables.
      2. Update multiple monitored tables using SQL (the update must be relatively large ~ 30K records per table).
      3. Immediately trigger blocking ad-hoc snapshot for the same tables updated in step 3 using file channel. Timing is important. 
      4. Immediately execute the same DML update as in step 3. Timing is important. 
      5. Due to race condition there will be multiple issues with the generated CDC stream. Specifically, snapshot and not snapshot events will be mixed together, not only in terms of order of events but also in terms of data (records which belong to different tables will have columns from each other).

      Feature request or enhancement

      For feature requests or enhancements, provide this information, please:

      Which use case/requirement will be addressed by the proposed feature?

      Fixing ad-hoc blocking snapshot for MySQL connector. Note: it seems all other connectors (except MySQL) don't have this issue. 

      Implementation ideas (optional)

      MySQL connector implements 'blocking' in the blocking snapshot by stopping current thread in io.debezium.connector.mysql.MySqlStreamingChangeEventSource.java until the snapshot is completed. 

       
      context.streamingPaused();

      context.waitSnapshotCompletion();

      Unfortunately change events are consumed in a separate 'binlog' thread by BinlogClient.EventsListener so it doesn't really stop the event handler (MySqlStreamingChangeEventSource#handleChange) from consuming inserts/updated/deletes during the blocking ad-hoc snapshot. As as result the queue includes multiple events in random order from multiple tables since snapshot and not snapshot events are populated in different threads running in parallel. 

      Suggestion: 

      Add blocking mechanizm to MySqlStreamingChangeEventSource#handleChange(...). Something as simple as the code below will works just fine:

       
      while (changeEventSourceContext.isPaused()) {

      Thread.sleep(100);

      }

       

       

              rh-ee-mvitale Mario Fiore Vitale
              maksym.sherbinin maksym Sherbinin (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

                Created:
                Updated:
                Resolved: