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

Defer transaction capture until the first DML event occurs

XMLWordPrintable

    • Icon: Enhancement Enhancement
    • Resolution: Done
    • Icon: Major Major
    • 2.6.0.Beta1
    • 2.5.1.Final
    • oracle-connector
    • None
    • False
    • None
    • False

      Bug report

      For bug reports, provide this information, please:

      What Debezium connector do you use and what version?

      debezium-connector-oracle version 2.5.1.Final

      What is the connector configuration?

      {
        "name": "source-test-connector",
        "config": {
          "connector.class": "io.debezium.connector.oracle.OracleConnector",
          "tasks.max": "1",
          "database.hostname": "oracle",
          "database.port": "1521",
          "database.user": "c##dbzuser",
          "database.password": "dbz",
          "database.dbname": "orclcdb",
          "database.pdb.name": "orclpdb1",
          "database.connection.adapter": "logminer",
          "topic.prefix": "dbz",
          "schema.name.adjustment.mode": "avro",
          "table.include.list": "C##DBZUSER.TEST_TABLE",
          "include.schema.changes": "false",
          "schema.history.internal.kafka.bootstrap.servers" : "kafka:9092",
          "schema.history.internal.kafka.topic": "schema-changes.test",
          "heartbeat.interval.ms": "60000",
          "log.mining.strategy": "online_catalog",
          "log.mining.query.filter.mode": "in",
          "custom.metric.tags": "connector=source-test-connector",
          "key.converter": "org.apache.kafka.connect.json.JsonConverter",
          "key.converter.schemas.enable": "true",
          "value.converter": "org.apache.kafka.connect.json.JsonConverter",
          "value.converter.schemas.enable": "true"
        }
      }
      

      What is the captured database version and mode of deployment?

      Oracle Database 19, Docker

      What behaviour do you expect?

      Only transactions that make changes to the tables we are interested in can affect connector offset SCN.

      What behaviour do you see?

      Offset SCN gets stuck due to any long transaction that does not even change the tables we are interested in.
      An alternative solution would be to use the following connector property:

      "log.mining.transaction.retention.ms": "600000",
      

      but this may have a negative impact on us and may result in data loss.

      I also noticed that long transactions lead to significant degradation of connector performance when lob.enabled=true because start SCN stops updating until the long transaction is completed.

      Example of SQL statements used by the connector to start LogMiner session:

      BEGIN sys.dbms_logmnr.start_logmnr(startScn => '2745794622950', endScn => '2745803960073', OPTIONS => DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG  + DBMS_LOGMNR.NO_ROWID_IN_STMT);END;
      BEGIN sys.dbms_logmnr.start_logmnr(startScn => '2745794622950', endScn => '2745805197949', OPTIONS => DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG  + DBMS_LOGMNR.NO_ROWID_IN_STMT);END;
      BEGIN sys.dbms_logmnr.start_logmnr(startScn => '2745794622950', endScn => '2745806641827', OPTIONS => DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG  + DBMS_LOGMNR.NO_ROWID_IN_STMT);END;
      

      where:
      startScn: 2745794622950 (2024-02-09 16:20:34)
      endScn:

      • 2745803960073 (2024-02-09 16:43:31)
      • 2745805197949 (2024-02-09 16:46:46)
      • 2745806641827 (2024-02-09 16:50:28)

      Thanks to question on Stack Overflow, I realized that this is done intentionally to correctly process data changes with CLOB/BLOB columns.
      But it seems that in the mode when LOB support is disabled, we may not handle the transaction start event at all, that is, we can defer transaction capture until the first DML event occurs.

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

      Yes

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

      [2024-01-17 18:07:25,312] WARN [source-test-connector|task-0] Transaction 7d011500732a2000 (start SCN 2687353029091, change time 2024-01-12T11:38:35Z, redo thread 1, 0 events) is being abandoned. (io.debezium.connector.oracle.logminer.processor.memory.MemoryLogMinerEventProcessor:155)
      

      Feature request or enhancement

      https://github.com/debezium/debezium/pull/5252

      I have prepared some changes, but I am having trouble changing the commit handling code.

      Implementation ideas (optional)

      I suggest removing the start operation from V$LOGMNR_CONTENTS view select query if LOB support is disabled.
      LogMinerEventProcessor is already partially ready for this, but it is necessary to prepare changes in the logic for handling commits.

              Unassigned Unassigned
              andrey.pustovetov@gmail.com Andrey Pustovetov (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

                Created:
                Updated:
                Resolved: