Source connector optimization

42 views
Skip to first unread message

Vladislav P

unread,
Dec 2, 2025, 7:39:38 AM (23 hours ago) Dec 2
to debezium
Безымянный.png
Hi everybody. Can you tell me how to optimize the connector so that the maximum read delay is no more than 3 minutes?

Now the delay can reach 2 hours, decrease to 30 minutes, and may not exceed 10 minutes at all. But the indicators are unstable.

The virtual machine has 16 GB of RAM installed, of which 9 GB is currently in use.

The connector is configured as follows:
{
"name": "{{sourceConnectorName}}",
"config": {
"connector.class": "io.debezium.connector.oracle.OracleConnector",
"tasks.max": "1",
"database.hostname": "{{sourceDatabaseHost}}",
"database.port": "{{sourceDatabasePort}}",
"database.user": "{{sourceDatabaseUser}}",
"database.password": "{{sourceDatabasePassword}}",
"database.dbname": "{{sourceDatabaseName}}",
"table.include.list": "... 33 tables",
"topic.prefix": "{{topicPrefix}}",
"database.server.name": "{{topicPrefix}}",
"schema.history.internal.kafka.topic": "dbz_oracle_wpms_history",
"schema.history.internal.kafka.bootstrap.servers": "{{kafkaBootstrapServers}}",

"message.key.columns": "...",

"key.converter": "io.apicurio.registry.utils.converter.AvroConverter",
"key.converter.apicurio.registry.url": "{{apicurioRegistryUrl}}",
"key.converter.apicurio.registry.auto-register": "true",
"key.converter.apicurio.registry.find-latest": "true",
"key.converter.schemas.enable": "false",
"key.converter.apicurio.registry.headers.enabled": "false",
"key.converter.apicurio.registry.as-confluent": "true",
"key.converter.apicurio.use-id": "contentId",
"value.converter": "io.apicurio.registry.utils.converter.AvroConverter",
"value.converter.apicurio.registry.url": "{{apicurioRegistryUrl}}",
"value.converter.apicurio.registry.auto-register": "true",
"value.converter.apicurio.registry.find-latest": "true",
"value.converter.schemas.enable": "false",
"value.converter.apicurio.registry.headers.enabled": "false",
"value.converter.apicurio.registry.as-confluent": "true",
"value.converter.apicurio.use-id": "contentId",
"schema.name.adjustment.mode": "avro",

"header.converter": "org.apache.kafka.connect.json.JsonConverter",
"header.converter.schemas.enable": "true",

"signal.enable.channels": "source",
"signal.data.collection": "DBNAME.SCHEMA.DEBEZIUM_SIGNAL",

"topic.creation.enable": "true",
"topic.creation.default.replication.factor": 1,
"topic.creation.default.partitions": 5,
"topic.creation.default.retention.ms": 345600000,
"topic.creation.default.cleanup.policy": "delete",

"log.mining.strategy": "hybrid",
"log.mining.query.filter.mode": "in",
"log.mining.batch.size.max": 6000000,
"log.mining.batch.size.default": 1200000,
"log.mining.transaction.retention.ms": 900000,
"log.cleanup.policy": "delete",
"log.retention.ms": 345600000,

"max.queue.size": 393216,
"max.batch.size": 98304,
"poll.interval.ms": 5,

"incremental.snapshot.chunk.size": 50000,
"incremental.snapshot.allow.schema.changes": "true",
"snapshot.fetch.size": 50000,
"query.fetch.size": 50000,

"snapshot.mode": "no_data",
"schema.history.internal.store.only.captured.tables.ddl": "true",
"snapshot.database.errors.max.retries": 2,
"internal.log.mining.log.query.max.retries": 15,

"heartbeat.interval.ms": "10000",
"heartbeat.action.query": "MERGE INTO WPMS.DEBEZIUM_HEARTBEAT t USING (SELECT 1 id, CURRENT_TIMESTAMP ts FROM dual) s ON (t.id = s.id) WHEN MATCHED THEN UPDATE SET t.ts = s.ts WHEN NOT MATCHED THEN INSERT (id, ts) VALUES (s.id, s.ts)",

"notification.enabled.channels": "sink,jmx,log",
"notification.sink.topic.name": "debezium_notifications"
}
}

Chris Cranford

unread,
Dec 2, 2025, 12:58:05 PM (17 hours ago) Dec 2
to debe...@googlegroups.com
Hi -

I'm afraid after looking at your metrics, it's highly unlikely to reach a 3 minute latency with your activity volume without seriously evaluating additional optimizations like filtering by users or by client ids, or what likely may help the most is seeing if whatever processes are generating so much data can be done in a more efficient way to reduce the redo they're creating.

But the variance in the latency is going to depend solely on the number of logs that need to be mined in a mining step. If you see MinimumMinedLogCount and MaximumMinedLogCount, this varies from 2 to 109. The more logs that need to be mined, the longer a session will take because there is significantly more bytes of data that need to be decoded, sorted, and then provided to Debezium by LogMiner. LogMiner is a single threaded database process, so having such a variance directly impacts its ability to provide fast results.

A rough estimate of this connector, it has been up for approximately 26-27 hours based on TotalBatchProcessingTimeInMilliseconds, and during this period you have had 212M transactions and you've processed nearly 2B rows of data from LogMiner (approximately 20k/second read but only consuming 1.5k/second), emitting approximately 142M changes out of nearly 2B. 

In terms of metrics that stand out 
  • There are 41 minutes spent parsing SQL statements (TotalParseTimeInMilliseconds). You've processed nearly 142M DML changes, but this seems like an unusually high amount of overall time within a day to parse SQL. Do you have tables with many columns and if so, what is your widest table?
  • There has been 8.1 hours spent waiting on calls to `ResultSet#next`, which is where we ask the JDBC driver to give us the next event to process.  Have you plotted the `TotalResultSetNextTimeInMilliseconds` over the day and seen spikes in its increase? If so, perhaps `query.fetch.size` is too small, and you could increase it to reduce those spikes.
  • You have at least one transaction that has been running for 24.4 minutes, based on `OldestScnAgeInMilliseconds`, but despite this only 67k events buffer. 
  • The maximum fetch query time is 55s, which could be part of the problem. The maximum batch size is 6M, and the query will only return once it has satisfied collecting data from LogMiner within that 6M span window. This means potentially more logs that must be read across that window, thus being slower due to the high read IO required.  Is 6M really necessary?
  • The log switch count is also questionable. This means you have about 20+ log switches per hour on average, which is 3-4x what Oracle recommends. Now its possible these switches are clustered in a small data load/batch process window and not indicative of switches over a full day, which means during those high periods of activity, there is higher LogMiner thrashing that happens tearing down and recreating the mining session. Do you have the raw numbers of of log switches by hour.
This Oracle database is quite busy, so additional to my earlier questions, is there any possibility for activity at the database to be filtered by other means, such as by specific database user accounts or by specific JDBC / Session clients?  

-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/6e3e9a2a-7223-446e-b9ee-29892a70bdc2n%40googlegroups.com.

Vladislav P

unread,
Dec 2, 2025, 5:58:06 PM (12 hours ago) Dec 2
to debezium
Hi Chris,

Thank you for your detailed analysis. Here are my responses to your questions:

1. Filtering possibilities:
Unfortunately, filtering by specific database users or client sessions is not feasible in our setup. We need to capture all activity from the monitored tables.

2. SQL parsing time and table structure:
The maximum number of columns read by the source connector is 11. All 33 tables.
However, in the original source db tables, some tables have up to 60 columns.
Regarding your question about high SQL parsing time (41 minutes): We do use a date column as part of a primary key that changes frequently. When this happens, we send 3 messages to Kafka (metadata, DELETE of old record, and INSERT of new record). Could this be contributing to the parsing overhead?

3. Batch size concerns:
You asked: "Is 6M really necessary?"
Our observation: When we reduce log.mining.batch.size.max and log.mining.batch.size.default, the metrics LagFromSourceInMilliseconds and MilliSecondsBehindSource gradually increase to 2+ hours. Should we not rely on these metrics?
Current behavior with our settings:
CommitThroughput reaches ~7000 but then drops when lag decreases.
The lag metrics are unstable. 

4. Current experiments:
We're testing with increased batch sizes:
"log.mining.batch.size.max": 15000000,
"log.mining.batch.size.default": 5000000
Results after 1 hour:
- LagFromSourceInMilliseconds reduced to ~150000 ms (2.5 minutes)
- CommitThroughput: 2800
- But then both lag metrics started increasing again
- TotalResultSetNextTimeInMilliseconds is at 1,534,467 ms and steadily growing

5. I'll try to find out about switching logs.
вторник, 2 декабря 2025 г. в 21:58:05 UTC+4, Chris Cranford:

Chris Cranford

unread,
Dec 2, 2025, 8:48:36 PM (9 hours ago) Dec 2
to debe...@googlegroups.com
Hi,

Regarding (2), the number of events or the date primary key shouldn't influence this. This would be a situation where either (a) there are tables with many columns or (b) where event SQL that is generated is quite large, i.e. 32k characters or more.

Regarding (3), the lag metrics themselves are not necessarily unstable, it's a product of the transaction volume patterns. If you have a batch job that loads 50M rows in a tight loop, many of those changes are going to be assigned the same SCN. The more events that are assigned the same SCN within a mining window create clustered regions of changes that need to be sorted by LogMiner using more expensive means to guarantee it provides Debezium changes in the right order. When the SCN is different, these expensive sort operations are unnecessary. So where you see higher lag periods are where LogMiner is forced to do more expensive work during these high activity periods, that create back pressure on how fast Oracle provides Debezium with the changes. 

In addition, during high windows of activity, you typically have more logs that need to mined versus a normal load window. If we go back to the 2 vs 109 for the number of logs mined in a single mining step, if those logs are 5GB each, you're talking about one step where we mined between 5GB-10GB of data and another where we mined potentially 540-545GB of data. That drastic of a switch in log switches impacts LogMiner's performance.

So a more accurate statement would be your transaction volume and patterns are what vary, and that translates to varied latency.

Regarding (4), what you need to be careful about is exhausting Oracle's SGA potentially leading to PGA_AGGREGATE_LIMIT exceptions or potentially OutOfMemoryError exceptions on the connector side. I've mentioned this on countless occasions, Oracle is permitted to assign 96k unique changes to the same SCN. And if these changes involve LOB or any non-data file storage operation (storing XML, extended strings, BLOB, CLOB, or JSON data as an example), the number of JDBC rows returned that have the same SCN can skyrocket. This is what I mean by clustered SCN scenarios, where you have spikes the number of change events that all share the same SCN. 

With that in mind, when setting the max batch to 15M and if we reach that number, that's potentially 1.44 trillion or more changes that can be in a single pass. While most Oracle systems currently cannot achieve the max of 96k yet, 20-30k is within reach, so that's still potentially 450 billion or more. Using such a max value I can only say is use at your own risk. It's not recommended under any circumstance.

If I were to be frank, I suspect your lag dropped simply due to your transaction activity/volume patterns. That's a data point that's really critical for tuning Debezium, and if you don't have a good understanding, modifying these parameters could be a hit or a miss. 

-cc

Vladislav P

unread,
1:13 AM (5 hours ago) 1:13 AM
to debezium
Hi,

If I understand correctly, then the default values for these parameters are more than correct for Debezium 3.3 to work, right?

"log.mining.batch.size.max": 100000,
"log.mining.batch.size.default": 20000.

But then what parameters can be improved to achieve a better result?
Monitor the "TotalResultSetNextTimeInMilliseconds" metric and increase the "query.fetch.size"?

Or is it necessary to go in a completely different direction?

среда, 3 декабря 2025 г. в 05:48:36 UTC+4, Chris Cranford:

Vladislav P

unread,
1:41 AM (4 hours ago) 1:41 AM
to debezium
We managed to get statistics on switching the log and it turned out to be sad

среда, 3 декабря 2025 г. в 10:13:56 UTC+4, Vladislav P:
redo_stats.txt
Reply all
Reply to author
Forward
0 new messages