oracle changes lost when using debezium oracle plugin

892 views
Skip to first unread message

Li Darren

unread,
Nov 28, 2023, 3:35:18 AM11/28/23
to debezium
Hi, there,
    we are using kafka connect debezium oracle plugin to capture source table changes .
below are the configurations.
{
    "connector.class": "io.debezium.connector.oracle.OracleConnector",
    "database.dbname": "xxx",
    "database.hostname": "xx.xx.xx.xx",
    "database.password": "*******",
    "database.port": "1521",
    "database.user": "xxxx",
    "log.mining.batch.size.default": "40000",
    "log.mining.batch.size.max": "200000",
    "log.mining.batch.size.min": "5000",
    "log.mining.continuous.mine": "true",
    "log.mining.query.filter.mode": "in",
    "log.mining.sleep.time.default.ms": "500",
    "log.mining.sleep.time.increment.ms": "500",
    "log.mining.sleep.time.max.ms": "1000",
    "log.mining.sleep.time.min.ms": "0",
    "log.mining.strategy": "online_catalog",
    "log.mining.username.include.list": "xxxx",
    "message.key.columns": "xxxxxxxxxx",
    "name": "14debezium_prod_used_source10",
    "schema.history.internal.kafka.bootstrap.servers": "192.168.1.8:9092, 192.168.1.9:9092, 192.168.1.9 :9092",
    "schema.history.internal.kafka.topic": "14debezium-schema-changes2",
    "schema.history.internal.store.only.captured.databases.ddl": "true",
    "schema.history.internal.store.only.captured.tables.ddl": "true",
    "schema.include.list": "xxxxx",
    "snapshot.max.threads": "8",
    "snapshot.mode": "schema_only",
    "table.include.list": "yyyyyyyy",
    "tasks.max": "1",
    "topic.prefix": "14DEBEZIUM2"
}
but we found some of the changes happened in source db are missing when above connect task finished.

pls provie some advice or clue to resolve the change records missing issue. Thanks 

Chris Cranford

unread,
Nov 28, 2023, 9:29:22 AM11/28/23
to debe...@googlegroups.com
Hi Li -

What version of Debezium are you using at this time?  I would suggest moving to 2.4.2.Final when it comes out or Debezium 2.5.0.Beta1 when its released this week.  There was a bug that we discovered that could cause some events to be missed that could be the cause for this problem that we rectified.

Thanks,
Chris
--
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 on the web visit https://groups.google.com/d/msgid/debezium/d2671eb3-f53e-485f-8df1-352644be46d9n%40googlegroups.com.

Li Darren

unread,
Nov 30, 2023, 1:23:28 AM11/30/23
to debezium
Hi, Chris,

   Thanks for the quick response.

   Currently, we are using debezium kafka connect oracle plugin Ver 2.4.0.Final with Oracle 11.0.2.4.

   And we find the latest version is 2.4.1.Final available on https://debezium.io/releases/2.4/.

   My Colleagure tried the new version and encountered issue/failure when Add/Delete connector via REST call. And tasks faild to run.

   Please let us know if you need more info/details. Thanks.

Chris Cranford

unread,
Nov 30, 2023, 2:25:36 AM11/30/23
to debe...@googlegroups.com
Hi Li -

Could you provide what the failure was when you tried to use 2.4.1?  The logs would be great if you have those to share.

Thanks,
Chris

Li Darren

unread,
Dec 1, 2023, 2:56:45 AM12/1/23
to debezium
Hi, Chris
  
  Below are the excerpt from connect.log file showing the exceptions when upgraded to 2.4.1.Final.
----------------------------------------------------------------------------------------------------------------------
   [2023-12-01 10:12:06,509] ERROR [14debezium_prod_used_source18|task-0] Mining session stopped due to error. (io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource:260)
java.lang.IllegalStateException: None of log files contains offset SCN: 416890767466, re-snapshot is required.
at io.debezium.connector.oracle.logminer.LogMinerHelper.setLogFilesForMining(LogMinerHelper.java:101)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.initializeRedoLogsForMining(LogMinerStreamingChangeEventSource.java:409)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:173)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)
at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:272)
at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:197)
at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:137)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
[2023-12-01 10:12:06,510] ERROR [14debezium_prod_used_source18|task-0] Producer failure (io.debezium.pipeline.ErrorHandler:52)
java.lang.IllegalStateException: None of log files contains offset SCN: 416890767466, re-snapshot is required.
at io.debezium.connector.oracle.logminer.LogMinerHelper.setLogFilesForMining(LogMinerHelper.java:101)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.initializeRedoLogsForMining(LogMinerStreamingChangeEventSource.java:409)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:173)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)
at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:272)
at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:197)
at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:137)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)


below are the connect configurations:
----------------------------------------------------------------------------------------------------------------------
curl -X POST http://localhost:8082/connectors -H "Content-Type: application/json" -d '{
    "name": "14debezium_prod_used_source19",
    "config": {
     "connector.class" : "io.debezium.connector.oracle.OracleConnector",
     "tasks.max" : "1",
     "database.dbname": "xxxx",
     "database.hostname": "xxxxx",

     "database.port": "1521",
     "database.user": "xxxx",
     "database.password": "YX5spph_wX^5u",
     "topic.prefix": "14DEBEZIUM12",
     "table.include.list": "cm_acmu_hist",
     "schema.history.internal.kafka.bootstrap.servers" : "172.27.102.15:9092,172.27.102.16:9092,172.27.102.17:9092",
     "schema.history.internal.kafka.topic": "14debezium-schema-changes12",
     "message.key.columns": "cm_acmu_hist:acct_id;cm_acmu_hist:sa_id",
     "snapshot.mode":"schema_only",
     "snapshot.max.threads": "8",
     "log.mining.strategy": "online_catalog",
     "log.mining.username.include.list":"abc",
     "schema.include.list":"abc",
     "schema.history.internal.store.only.captured.tables.ddl": true,
     "schema.history.internal.store.only.captured.databases.ddl": true,
     "log.mining.query.filter.mode": "in",
     "log.mining.batch.size.min": 5000,
     "log.mining.batch.size.max": 200000,
     "log.mining.batch.size.default": 50000,
     "log.mining.sleep.time.min.ms": 0,
     "log.mining.sleep.time.max.ms": 1000,
     "log.mining.sleep.time.default.ms": 500,
     "log.mining.sleep.time.increment.ms": 500,
     "log.mining.transaction.retention.ms": 3600000
  }
}'

connect_debezium_oracle.log

Li Darren

unread,
Dec 1, 2023, 5:41:31 AM12/1/23
to debezium
And BTW,  the source Oracle is 11.2.0.4 RAC

Li Darren

unread,
Dec 1, 2023, 7:30:15 AM12/1/23
to debezium
{
  "name": "14debezium_prod_used_source21",
  "connector": {
    "state": "RUNNING",
    "worker_id": "0.0.0.0:8082"
  },
  "tasks": [
    {
      "id": 0,
      "state": "FAILED",
      "worker_id": "0.0.0.0:8082",
      "trace": "org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:262)

at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)
at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:272)
at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:197)
at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:137)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)\nCaused by: java.lang.IllegalStateException: None of log files contains offset SCN: 416935741690, re-snapshot is required.

at io.debezium.connector.oracle.logminer.LogMinerHelper.setLogFilesForMining(LogMinerHelper.java:101)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.initializeRedoLogsForMining(LogMinerStreamingChangeEventSource.java:409)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:173)
... 9 more\n"
    }
  ],
  "type": "source"

Li Darren

unread,
Dec 1, 2023, 7:33:39 AM12/1/23
to debezium
guess we're encountering some sequence gap here.
log_err_20231201203212.png

Chris Cranford

unread,
Dec 1, 2023, 10:18:47 AM12/1/23
to debe...@googlegroups.com
Hi Li -

Can you please execute the following queries and provide the output of the data-set:

SELECT NAME, THREAD#, SEQUENCE#, FIRST_CHANGE#, NEXT_CHANGE#
FROM V$ARCHIVED_LOG
WHERE 416890767466 >= FIRST_CHANGE#

SELECT LF.MEMBER, LF.GROUP#, L.THREAD#, L.SEQUENCE#, L.STATUS, L.FIRST_CHANGE#, L.NEXT_CHANGE#
FROM V$LOG L, V$LOGFILE LF
WHERE LF.GROUP# = L.GROUP#
AND 416890767466 >= L.FIRST_CHANGE#

This should provide me with all the details of the logs that include and come after SCN 416890767466 to see whether there is a gap or some data state that is unexpected.

Thanks,
Chris

Li Darren

unread,
Dec 1, 2023, 11:57:59 AM12/1/23
to debezium
Hi, Chris,

    The 1st query's resuls are enclosed.

     2nd query returns nothing.

1st_output.csv

Li Darren

unread,
Dec 3, 2023, 10:45:19 PM12/3/23
to debezium
Hi, Chris, 

   Pls let me know if you need more info. Thx.

Chris Cranford

unread,
Dec 4, 2023, 7:31:21 AM12/4/23
to debe...@googlegroups.com
Hi Li -

Can you rerun the second query but drop the join against V$LOGFILE and only grab the contents from the V$LOG table.

Thanks,
Chris

Chris Cranford

unread,
Dec 4, 2023, 7:34:58 AM12/4/23
to debe...@googlegroups.com
Hi Li -

Actually the data won't be necessary.  If you look at the first CSV file you provided, the first column is NULL for every entry, and that's a problem.  This means the file for that archive log is no longer available and when that happens, Debezium will have no way to read that data from disk.  That would explain why you received this error.  It would seem that perhaps you have a "too aggressive" archive log policy that removes archive logs.

Thanks,
Chris

Li Darren

unread,
Dec 5, 2023, 9:19:42 PM12/5/23
to debezium
Thanks, Chris. We'll work with our DBA on the  archive strategy. 

Marcelo Costa

unread,
Dec 5, 2023, 9:48:34 PM12/5/23
to debe...@googlegroups.com
Hello, to support the conversation, 

We adopted a strategy of keeping redo log files for at least three days. DBAs don't like it, and it is necessary to detail and explain the need.

Another critical factor is to consider the size of the redo log file so that it can be read quickly. Using smaller Redo Logs optimizes reading speed. Instead of having to scroll through a 4 GB file to read and collect just 1 MB, which is, in fact, the interest of collection, scroll through a 2 GB file and thus obtain the same result with better performance. This parameter is also configured in Logminer. 

Regards,
Marcelo Costa





Chris Cranford

unread,
Dec 6, 2023, 12:54:13 AM12/6/23
to debe...@googlegroups.com
HI Marcelo -

So recently I acknowledged a bug with the Oracle connector, let me try and detail out the bug and how this could have impacted your situation.

Since Oracle deprecated and removed access to the continuous mining feature in Oracle 19, the ownership of calculating the redo and archive logs that should be mined is up to the connector itself.  There was a bug where the list of logs was being incorrectly calculated in a specific situation where the state of data in the V$ARCHIVED_LOG, V$LOGFILE, and V$LOG metadata tables was inconsistent due to the processing speed of the ARC process.  This specific bug most often manifested in Oracle environments where the database is configured to use redo log filenames where the sequence or some other dynamic value causes the redo log filenames to be different when the log's sequence changes. If the connector attempted to gather the list of archive and redo logs to mine during this inconsistent window, this would then lead to several problems.

The most common and known issues with this bug was that

    1. Users would face "None of the logs contain the offset scn: XXXXXXXX".
    2. Users would report that certain transactions were not emitted or that some events were missing.

In Debezium 2.4.1.Final and 2.5.0.Alpha2, we first attempted to address the problem. In essence, because Oracle's metadata can be inconsistent, we had to introduce a retry mechanism to circumvent this inconsistency and recheck the metadata tables when we identified this condition to avoid the connector moving ahead with the mining.  In effect, if Debezium is waiting on the ARC process to move a redo log to the archive and register the entry in V$ARCHIVED_LOG, it will now do so.

Unfortunately the fix in 2.4.1.Final and 2.5.0.Alpha2 only satisfied Oracle Standalone or Oracle RAC environments with a single node (single redo thread).  Any environment that utilizes more than one redo thread would have likely quickly noticed that they were facing the same (1) error even when they checked the metadata tables and found the redo and archive logs existed.  This regression has been fixed in 2.5.0.Beta1 (released this week) and will be included in Debezium 2.4.2.Final (due out very soon).

Now I can't say whether or not your situation is attributed to this bug or not, but it's reasonable to think it does -- so I would upgrade ASAP.  My hope is that this should keep the archive log retention need for most environments at a reasonable level that most DBAs are willing to accept going forward :).

Also, just for completeness sake, if you find the retry logic for the logs isn't satisfactory, I believe the default is 5 retries before we fault, these can be tuned:

    internal.log.mining.query.max.retries
    Defaults to 5, the number of retries to attempt to get a full set of logs without sequence gaps.

    internal.log.mining.log.backoff.initial.delay.ms
    Defaults to 1000, the number of milliseconds to initially wait before retrying the log sequence gap sanity check
   
    internal.log.mining.log.backoff.max.delay.ms
    Defaults to 60000, the maximum number of milliseconds to wait before retrying the log sequence gap sanity check

The sanity-check fires quickly and then backs off exponentially from 1s up to 60s.

To your last point, I just want to mention that redo log file sizes do in fact matter.  I worked with one community member who had between 4GB and 6GB redo logs, they reduced them to 2-3GB each and they noticed a performance increase in their LogMiner throughput.  But this very often is dependent on the hardware and topology of your database.  For example, are the logs stored on ASM or the local disks of the server, etc.  But sizing your redo logs too small can have adverse impacts on your database's performance, particularly if you find you have a high burst of traffic that causes your logs to roll faster than your ARC process can maintain -- and this can lead to database deadlocks while the database waits (quite literally) for the ARC process to make a new redo log that isn't full available.

Let me know if you have any other questions.
Chris

Marcelo Costa

unread,
Dec 6, 2023, 12:44:50 PM12/6/23
to debe...@googlegroups.com
Hi Chris, good to see you here.

We faced these challenges described by you regarding data loss (in fact, the Logminer abandoned transactions) as well as the loss of events.

In our case, we created mechanisms to overcome this flaw, working in the way you described. Our version of Debezium is still 1.9.5, and I confess that we did a lot of tuning on it to meet our needs. We are aware that at some point (I hope it will be at the beginning of 2024), we will need to merge the main project with the tunings we have implemented here and thus collaborate with the main project if approved, of course.

Another problem we faced, which caused our performance to be deficient, concerned how the query in the V$LOGMNR_CONTENTS view was carried out. We changed this form of query to use some SQL clauses such as IN and EQUAL, and we got good numbers.

These actions allowed us to reach the number of TPS we currently have, where we can move more than 1TB per day in data from Oracle (both 11g and 19c), as well as reaching 300K TPS (in fact, this week, we reached a new record where we reached an incredible 336K TPS).

Many details need to be clarified regarding data collection, specifically in Oracle. Here, we managed to find a good balance that leaves us stable and without incidents. In SAP specifically, there are maintenance routines (REORG, for example) that hinder data collection and break out the CDC environment.

This week, we discovered a possible BUG related to Oracle RAC (with Global Cache and Interconnect specifically) when we have two nodes in the cluster. We are carrying out tests, but so far, our strategy has proven to be stable, and we have stopped having problems with the database environment.

We also needed to perform some tuning on the operating system as well as on the network interfaces, which also contributed to our performance reaching these TPS numbers.

We have learned that Oracle is one of the most challenging environments in which to do CDC. However, after almost 18 months on this journey, I understand that we have achieved the necessary stability and are now just cruise flights and implementing some fine-tuning.

Marcelo Costa
-------------------------------------------------
https://www.linkedin.com/in/marcelojscosta/




Chris Cranford

unread,
Dec 6, 2023, 3:19:27 PM12/6/23
to debe...@googlegroups.com, Marcelo Costa
Hi Marcelo

See some comments inline...

Thanks,
Chris

On 12/6/23 12:44, Marcelo Costa wrote:
Hi Chris, good to see you here.

We faced these challenges described by you regarding data loss (in fact, the Logminer abandoned transactions) as well as the loss of events.

In our case, we created mechanisms to overcome this flaw, working in the way you described. Our version of Debezium is still 1.9.5, and I confess that we did a lot of tuning on it to meet our needs. We are aware that at some point (I hope it will be at the beginning of 2024), we will need to merge the main project with the tunings we have implemented here and thus collaborate with the main project if approved, of course.

That would be wonderful, contributions are always welcomed :)


Another problem we faced, which caused our performance to be deficient, concerned how the query in the V$LOGMNR_CONTENTS view was carried out. We changed this form of query to use some SQL clauses such as IN and EQUAL, and we got good numbers.

We fixed this around Debezium 2.1 with the new `log.mining.query.filter.mode`. This mode allows you to set `none`, `in`, or `regex`.  The first does no database filtering and only applies the filtering on the connector side, the middle is as you likely suspect, it uses an IN-clause, and the latter reintroduces the REGEXP_LIKE operation for those who specify include/exclude lists with regular expressions.  I'd be interested in understanding your EQUAL solution to see how that really differs performance wise from IN.


These actions allowed us to reach the number of TPS we currently have, where we can move more than 1TB per day in data from Oracle (both 11g and 19c), as well as reaching 300K TPS (in fact, this week, we reached a new record where we reached an incredible 336K TPS).

Those are some crazy numbers.  Would you be willing to discuss offline some of your tuning techniques?  I'd love to do a deep-dive if you're open and willing, perhaps at the start of the new year?


Many details need to be clarified regarding data collection, specifically in Oracle. Here, we managed to find a good balance that leaves us stable and without incidents. In SAP specifically, there are maintenance routines (REORG, for example) that hinder data collection and break out the CDC environment.
Ya, SAP is notorious for that, I remember that vividly from a past job where we had SAP on SQL Server.
Reply all
Reply to author
Forward
0 new messages