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

Oracle Logminer: records missed during switch from snapshot to streaming mode

XMLWordPrintable

    • False
    • None
    • False
    • Hide
      1. Setup an Oracle DB configured for logminer, with a table that will be captured.
      2. Start inserting records in the database, one by one, each in its own transaction.
      3. While inserting records, create a connector.
      4. Check if all inserted records are captured by Debezium, one should be missing.

      (more details given in the issue)

      Show
      Setup an Oracle DB configured for logminer, with a table that will be captured. Start inserting records in the database, one by one, each in its own transaction. While inserting records, create a connector. Check if all inserted records are captured by Debezium, one should be missing. (more details given in the issue)

      Bug report

      This bug seems to be related to https://issues.redhat.com/browse/DBZ-4367.

      In short: it seems that it is still possible for changes to go missing during the switch from snapshot to streaming mode, though it is much harder to do than before.

      What Debezium connector do you use and what version?

      Oracle connector 1.9.1.Final

      What is the connector configuration?

      The connector configuration has been added as an attachment. Note that the we are using the embedded engine.

      What is the captured database version and mode of deployment?

      An Oracle 19c database, running locally in a docker container.

      What behaviour do you expect?

      All records inserted into the database should be captured.

      What behaviour do you see?

      A record is missing, right when the connector switches from snapshot to streaming mode.

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

      I haven't had the time yet to test with a 2.0.0 alpha release.

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

      The logs have been added as an attachment, the Debezium log level was on TRACE. Since we are using Debezium engine, there will be some logs in there from the host application as well.

      There was a lot of junk in these logs, so I pruned them a bit to make it possible to navigate them, for instance I removed the constant:

      2022-05-03 09:54:43,528 DEBUG [io.deb.con.bas.ChangeEventQueue] (executor-thread-17) checking for more records...
      2022-05-03 09:54:43,528 DEBUG [io.deb.con.bas.ChangeEventQueue] (executor-thread-17) no records available yet, sleeping a bit...
      

      Let me know if you think anything's missing, I'll add the full logs then.

      In this particular case, it is the record with COL1=273 that is missing. 272 was the last record of the snapshot,  274 the first record in streaming mode (see line 726 and onward).

      How to reproduce the issue using our tutorial deployment?

      In order to reproduce this, we need to insert some records from one thread, while creating the connector from another. Using the examples from the tutorial (after creating a table and adding the supplemental log data):

      Have one thread doing something like:

      for i in {1..30000}
      do
         echo "INSERT INTO test_table VALUES (${i}, 'hello');" | docker exec -i dbz_oracle sqlplus debezium/dbz@//localhost:1521/ORCLPDB1
        echo "commit;" | docker exec -i dbz_oracle sqlplus debezium/dbz@//localhost:1521/ORCLPDB1
      done
      

      While data is being inserted, create a connector

      curl -i -X POST -H "Accept:application/json" -H  "Content-Type:application/json" http://localhost:8083/connectors/ -d @register-oracle-logminer.json
      

      Important here is to commit every record separately, or at least in very small transactions. The larger the transactions, the less likely this bug is to occur.

      My guess at what's going on

      In OracleSnapshotChangeEventSource.java, in the determineSnapshotOffset method, we can see the following code:

              do {
                  currentScn = jdbcConnection.getCurrentScn();
              } while (areSameTimestamp(latestTableDdlScn.orElse(null), currentScn));
      
              // Record the starting SCNs for all currently in-progress transactions.
              // We should mine from the oldest still-reachable start SCN, but only for those transactions.
              // For everything else, we mine only from the snapshot SCN forward.
              Map<String, Scn> snapshotPendingTransactions = getSnapshotPendingTransactions(currentScn);
      

       
      If a transaction is committed between retrieving the currentScn and the call to getSnapshotPendingTransactions, those changes will not be captured.

       

      I might have a go at creating an integration test for this, as the issue is quite tricky to replicate.

        1. pruned-log-273
          4.63 MB
        2. connector-config
          2 kB
        3. dbz-diff
          2 kB
        4. trace-log
          26.31 MB

              ccranfor@redhat.com Chris Cranford
              robin-vanderstraeten-klarrio Robin Vanderstraeten (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

                Created:
                Updated:
                Resolved: