[Oracle Source] Not applying CDC changes via Redo Logs even though it is present.

17 views
Skip to first unread message

Ermano Winata

unread,
Jul 28, 2025, 11:53:49 AMJul 28
to debezium
Database Specs:
Oracle v11.2.0.2 on a Physical Standby Read only with apply. On a 2 node RAC

Never got streaming to work. It will keep polling with no data to stream to the topic even though the table is very actively adding rows (very frequent and high volume table inserts).

Schema topic gets created and DDL is parsed okay so that seems to eliminate the issue for wrong configs for:
- table.include.list
- schema.include.list
- database connection configs

Also configured to use archive only. Querying the archive log does show that the log at the recorded SCN for the offset is available. Not quite sure where the issue is here since the connector is still running after 4+ hours with no streaming data sent towards the topic.

Here is the config:

```
{
"name": "oracle-source-archive-1",
"connector.class": "io.debezium.connector.oracle.OracleConnector",
"tasks.max": "1",
"schema.history.internal.consumer.sasl.jaas.config": "org.apache.kafka.common.security.scram.ScramLoginModule required username=\"**\" password=\"**\";",
"schema.include.list": "**",
"internal.log.mining.read.only": "true",
"producer.override.security.protocol": "SASL_PLAINTEXT",
"log.mining.strategy": "online_catalog",
"schema.history.internal.store.only.captured.tables.ddl": "true",
"topic.prefix": "oracle.source.archive.1",
"schema.history.internal.kafka.topic": "_schemahistory.oracle.archive.1",
"schema.history.internal.producer.security.protocol": "SASL_PLAINTEXT",
"log.mining.archive.log.only.mode": "true",
"schema.history.internal.producer.sasl.mechanism": "SCRAM-SHA-256",
"database.user": "**",
"database.dbname": "**",
"database.connection.adapter": "logminer",
"schema.history.internal.kafka.bootstrap.servers": "kafka-controller-0.kafka-controller-headless.kafka.svc.cluster.local:9092,kafka-controller-1.kafka-controller-headless.kafka.svc.cluster.local:9092,kafka-controller-2.kafka-controller-headless.kafka.svc.cluster.local:9092",
"database.port": "1521",
"database.hostname": "**",
"database.password": "**",
"name": "oracle-source-archive-1",
"schema.history.internal.producer.sasl.jaas.config": "org.apache.kafka.common.security.scram.ScramLoginModule required username=\"**\" password=\"**\";",
"schema.history.internal.consumer.sasl.mechanism": "SCRAM-SHA-256",
"producer.override.sasl.jaas.config": "org.apache.kafka.common.security.scram.ScramLoginModule required username=\"**\" password=\"**\";",
"table.include.list": "**.**",
"producer.override.sasl.mechanism": "SCRAM-SHA-256",
"snapshot.mode": "no_data",
"schema.history.internal.consumer.security.protocol": "SASL_PLAINTEXT"
}
```

Chris Cranford

unread,
Jul 28, 2025, 12:34:47 PMJul 28
to debe...@googlegroups.com
Hi -

Can you please enable TRACE logging for `io.debezium.connector.oracle` and share the connector logs, please.

Thanks
-cc
``` --
You received this message because you are subscribed to the Google Groups "debezium" group.
To unsubscribe from this group and stop receiving emails from it, send an email to debezium+u...@googlegroups.com.
To view this discussion visit https://groups.google.com/d/msgid/debezium/263b00e3-6c75-44da-8e00-954f159e2306n%40googlegroups.com.

Ermano Winata

unread,
Jul 28, 2025, 1:43:16 PMJul 28
to debezium
Here is the trace logs
kafka-logs.log

Chris Cranford

unread,
Jul 28, 2025, 6:48:07 PMJul 28
to debe...@googlegroups.com
Hi -

So I would say based on the archive log details in the connector log, your DBA has performed one of several tasks in your environment:

    - Restored the database from an earlier point in time
    - Restored some archive logs from another system for analysis

We can determine that something is off by looking at the SCN ranges associated with the archive logs versus the online log redos. For example, for redo thread 1, the current online log sequence 66522 has a last flushed redo SCN of
48390191574883, but this comes before the eldest archive log for the same redo thread with sequence 66508, which starts at SCN 48757761185298.  The same applies to redo threads 2 and 3.

If your DBA has recently done a restore, typically they should force Oracle to have a new log incarnation ID. If you're on a Debezium version prior to 3.1.2.Final / 3.2.0.Beta1, that incarnation ID was not respected, and we would mistakenly fetch logs for the older incarnation by mistake. If you are on such an older Debezium version, please update and restart the connector and that solve the issue.

If your DBA has recently restored some logs for analysis, they generally would do this by adding them to a secondary destination location rather than the main location that the database is currently writing to. When more than one local/valid archive destination is configured, Debezium picks the first row returned and uses that DEST_ID, which may not be the one that the system is currently writing to. In such cases, it's critical that Debezium is configured with `archive.destination.name` (`log.mining.archive.destination.name` on older versions) so that Debezium sources the proper log catalog entries.

If neither of those cases hold true, then I would recommend asking the DBA team to explain why the archive logs have SCN ranges that are larger than the online redo logs, as that's not expected. In this case, its advised that

    - The DBA remove the old log files to avoid confusing Debezium
    - Consider removing the offsets/history topic and redeploy the connector taking a new snapshot

Let us know if you have any other questions.
-cc

On 7/28/25 1:43 PM, 'Ermano Winata' via debezium wrote:
Here is the trace logs --
You received this message because you are subscribed to the Google Groups "debezium" group.
To unsubscribe from this group and stop receiving emails from it, send an email to debezium+u...@googlegroups.com.
Reply all
Reply to author
Forward
0 new messages