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

Missing oracle cdc records

    XMLWordPrintable

Details

    • False
    • None
    • False

    Description

      i noticed around 5000 oracle cdc records are not streaming to kafka for 2 tables, but later rows from those tables have streamed to kafka

      when i check logs i notice that offset scn value has not advanced for around 15 hours
      logs also have:

      DEBUG Detected transaction with null username MemoryTransaction{numberOfEvents=7408} AbstractTransaction{transactionId='61001b00ad881b00', startScn=527832412914, changeTime=2022-10-20T07:53:07Z, userName='null'} (io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor)
      

      i have attached some of v$logmnr_contents too (which shows a couple of the rows that are missing), one thing that appears odd is start_timestamp has year 1988, and username of UNKNOWN
      note: that the sample missing row was created in the oracle db at 2022-10-20 07:52:37

      looking at the code https://github.com/debezium/debezium/blob/a0f3263fa60b95d5b10b0c8ce10d4a8bbcf028ba/debezium-connector-oracle/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java#L446 i wonder if its possible that 'getTransactionEventCount(transaction) > 0' condition is not met if the first event in a transaction is in progress of being evaluated could the count be 0? similarly https://github.com/debezium/debezium/blob/a0f3263fa60b95d5b10b0c8ce10d4a8bbcf028ba/debezium-connector-oracle/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java#L512 might not return at all if eventcount=0

       

      SCN START_SCN COMMIT_SCN TIMESTAMP START_TIMESTAMP COMMIT_TIMESTAMP XIDUSN XIDSLT XIDSQN XID PXIDUSN PXIDSLT PXIDSQN PXID TX_NAME OPERATION OPERATION_CODE ROLLBACK SEG_OWNER SEG_NAME TABLE_NAME SEG_TYPE SEG_TYPE_NAME TABLE_SPACE ROW_ID USERNAME OS_USERNAME MACHINE_NAME AUDIT_SESSIONID SESSION# SERIAL# SESSION_INFO THREAD# SEQUENCE# RBASQN RBABLK RBABYTE UBAFIL UBABLK UBAREC UBASQN ABS_FILE# REL_FILE# DATA_BLK# DATA_OBJ# DATA_OBJV# DATA_OBJD# SQL_REDO SQL_UNDO RS_ID SSN CSF INFO STATUS REDO_VALUE UNDO_VALUE SAFE_RESUME_SCN CSCN OBJECT_ID EDITION_NAME CLIENT_ID SRC_CON_NAME SRC_CON_ID SRC_CON_UID SRC_CON_DBID SRC_CON_GUID CON_ID
      527832407719 0 527832407719 2022-10-20 07:52:40 1988-01-01 00:00:00 2022-10-20 07:52:40 103 13 1972559 67000D004F191E00 103 13 1972559 67000D004F191E00   COMMIT 7 0       0     AAAAAAAAAAAAAAAAAA UNKNOWN UNKNOWN UNKNOWN 0 0 0 UNKNOWN 1 1 47092 21221 468 11 0 0 0 11 0 0 0 0 0 commit;    0x00b7f4.000052e5.01d4  0 0   0 305738 305739 527832407719 527832407719     UNKNOWN iredacted 3 3867249061 0   1
      527832407261     2022-10-20 07:52:37     103 13 1972559 67000D004F191E00 103 13 1972559 67000D004F191E00   INSERT 1 0 iredacted iredacted iredacted 2 TABLE iredacted AALKIPADtAAEiJEAAK UNKNOWN UNKNOWN UNKNOWN 0 0 0 UNKNOWN 1 1 47092 4 144 11 999416 56 17347 29 237 1188420 292459 1 2925071 insert into iredacted; delete from iredacted;  0x00b7f4.00000004.0090  10 0   0 144350 144351         UNKNOWN iredacted 3 3867249061 0   1

       

      larger log snippet:

      [2022-10-20 07:53:04,626] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:53:09,793] DEBUG Fetching results for SCN [527832385131, 527832412994] (io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor)
      [2022-10-20 07:53:14,626] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:53:24,626] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:53:34,629] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:53:44,632] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:53:54,634] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:54:04,637] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:54:14,640] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:54:24,643] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:54:34,645] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:54:44,648] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:54:45,733] DEBUG Counters{rows=149391, stuckCount=1, dmlCount=132081, ddlCount=0, insertCount=112067, updateCount=20014, deleteCount=0, commitCount=8650, rollbackCount=4, tableMetadataCount=0}. (io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor)
      [2022-10-20 07:54:45,733] DEBUG Processed in 95823 ms. Lag: 97724. Offset SCN: 527831752573, Offset Commit SCN: CommitScn [redoThreadCommitScns={1=RedoThreadCommitScn{thread=1, commitScn=527832405961, txIds=[16000c0019e53400]}}], Active Transactions: 7, Sleep: 60000 (io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor)
      [2022-10-20 07:54:54,651] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:55:04,651] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:55:14,651] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:55:24,651] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:55:34,652] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:55:44,652] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:55:46,652] DEBUG Fetching results for SCN [527832412989, 527832453449] (io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor)
      [2022-10-20 07:55:52,067] DEBUG Detected transaction with null username MemoryTransaction{numberOfEvents=7408} AbstractTransaction{transactionId='61001b00ad881b00', startScn=527832412914, changeTime=2022-10-20T07:53:07Z, userName='null'} (io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor)
      [2022-10-20 07:55:54,652] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:56:04,654] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:56:14,655] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:56:24,657] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:56:34,660] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:56:44,666] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:56:54,669] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:57:04,671] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:57:14,674] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:57:24,677] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:57:34,679] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:57:44,682] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 68 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:57:54,689] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:58:04,692] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:58:14,695] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:58:24,697] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:58:34,700] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:58:44,703] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:58:54,706] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:59:04,709] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 265 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:59:14,715] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 152 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2022-10-20 07:59:21,302] DEBUG Counters{rows=307943, stuckCount=1, dmlCount=275865, ddlCount=0, insertCount=230527, updateCount=45338, deleteCount=0, commitCount=16036, rollbackCount=2, tableMetadataCount=0}. (io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor)
      [2022-10-20 07:59:21,302] DEBUG Processed in 214591 ms. Lag: 228301. Offset SCN: 527831752573, Offset Commit SCN: CommitScn [redoThreadCommitScns={1=RedoThreadCommitScn{thread=1, commitScn=527832450247, txIds=[64002000e8243800]}}], Active Transactions: 8, Sleep: 57000 (io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor)
      [2022-10-20 07:59:24,723] INFO WorkerSourceTask{id=kafka-connect-redact-01-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask) 

       

      cc ccranfor@redhat.com

      Attachments

        Issue Links

          Activity

            People

              ccranfor@redhat.com Chris Cranford
              tooptoop toop toop (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: