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

Oracle Logminer: snapshot->stream switch misses DB changes in ongoing transactions

    XMLWordPrintable

Details

    • False
    • False
    • Hide

      In Oracle 19c

      -- the steps below were done through SQL*Plus, which does not auto-commit
      create table txtest(id int primary key, val varchar2(100));
      grant select on txtest to c##dbzuser;
      alter table txtest add supplemental log data (all) columns;
      insert into txtest values (1, 'pre-snapshot');
      commit;
      -- start a new transaction here; this one straddles the snapshot->streaming switch
      insert into txtest values(2, 'in tx, pre-snapshot');
      -- Now set up a debezium connector for table txtest, wait for the snapshot phase to complete
      -- as expected only row 1 (pre-snapshot) appears on Kafka
      insert into txtest values(3, 'in tx, post-snapshot');
      commit;
      -- this transaction is fully handled in streaming mode
      insert into txtest values (4, 'post-tx, streaming');
      commit;
      

      Debezium will only output change records for rows

      • 1 (TX completed pre-snapshot)
      • 3 (TX that spans snapshot->streaming switch: Debezium only records post-switchover changes)
      • 4 (TX lifetime was completely in the streaming phase)

      The change record for row 2 is never emitted, simply because the logs are never mined that far back.

      I have attached the Kafka Connect logs captured during the reproduction scenario (io.debezium.connector.oracle.logminer logs at TRACE level, the rest at INFO level). The salient bits start around line 2127.

      Show
      In Oracle 19c -- the steps below were done through SQL *Plus, which does not auto- commit create table txtest(id int primary key , val varchar2 (100)); grant select on txtest to c ##dbzuser; alter table txtest add supplemental log data ( all ) columns ; insert into txtest values (1, 'pre-snapshot' ); commit ; -- start a new transaction here; this one straddles the snapshot->streaming switch insert into txtest values (2, ' in tx, pre-snapshot' ); -- Now set up a debezium connector for table txtest, wait for the snapshot phase to complete -- as expected only row 1 (pre-snapshot) appears on Kafka insert into txtest values (3, ' in tx, post-snapshot' ); commit ; -- this transaction is fully handled in streaming mode insert into txtest values (4, 'post-tx, streaming' ); commit ; Debezium will only output change records for rows 1 (TX completed pre-snapshot) 3 (TX that spans snapshot->streaming switch: Debezium only records post-switchover changes) 4 (TX lifetime was completely in the streaming phase) The change record for row 2 is never emitted, simply because the logs are never mined that far back. I have attached the Kafka Connect logs captured during the reproduction scenario (io.debezium.connector.oracle.logminer logs at TRACE level, the rest at INFO level). The salient bits start around line 2127.

    Description

      Debezium may miss some changes to a table, if they are done

      • in a TX that was opened before the connector is started
      • the change is done before the connector is started
      • where that TX is committed only after the connector has started its initial consistent snapshot phase

      What goes wrong:

      Debezium/Logminer only mines SCNs >= the SCN at which the initial consistent snapshot phase operates. Let's call that SCN "Ts".

      If you open a transaction before the connector is started, and commit it after the snapshot phase has begun, you'll have a series of SCNs in the snapshot that look something like this:

      • Ts - 10 (INSERT prior to start of connector)
      • Ts - 9 (UPDATE prior to start of connector)
      • Ts + 5 (INSERT after the connector has started snapshotting)
      • Ts + 10 (TX COMMIT)

      Because the connector in streaming mode disregards all SCNs < Ts, its understanding of the transaction will consist of only SCNs (Ts + 5, Ts + 10). The changes made at Ts - 10 and Ts - 9, which come into effect due to the COMMIT, are never captured by the connector, and hence never emitted to Kafka.

      For transactions that straddle the snapshot->streaming switchover, the connector should mine back to the beginning of the TX, the snapshot SCN is not the right cutoff point there.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              dominique.chanet@klarrio.com Dominique Chanet (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: