Max Batch size too small even with a huge value

250 views
Skip to first unread message

Lucas Rangel Gazire

unread,
Jul 18, 2024, 12:10:19 PM7/18/24
to debezium
Hi!
I'm running in debug mode my embedded instance and notice that after a while i get the following message:

{"time":"2024-07-18T13:04:44.304-03:00","message":"Max batch size too small, using current SCN 161955624544 as end SCN.","logger_name":"io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource","thread_name":"debezium-oracleconnector-mfi.debezium-change-event-source-coordinator","level":"DEBUG","dbz.connectorName":"mfi.debezium","dbz.databaseName":"DATABASE","dbz.connectorType":"Oracle","dbz.taskId":"0","dbz.connectorContext":"streaming"}
From what i understand this can cause a lag in my data streaming because i can't follow the current SCN in time. Is there a calculation to do to make this value more assertive?

Currently the max batch size and queue size have an huge value:
DEBEZIUM_FIXED_INCOME_MIN_BATCH_SIZE: 5000000
DEBEZIUM_FIXED_INCOME_MAX_BATCH_SIZE: 200000000
DEBEZIUM_FIXED_INCOME_DEFAULT_BATCH_SIZE: 50000000
DEBEZIUM_FIXED_INCOME_MAX_QUEUE_SIZE: 210000000

Chris Cranford

unread,
Jul 18, 2024, 1:00:26 PM7/18/24
to debe...@googlegroups.com
Hi Lucas -

I think there may be some confusion.

Debezium out of the box supports a "max.queue.size" and "max.batch.size" configuration, but these are applicable to the internal buffer that sits between the runtime (Debezium Server/Kafka Connect/Embedded Engine) and the connector.  Oracle is a bit unique in that it maintains a sliding batch window configuration so that as the connector gets closer to near real-time the batch size is smaller and as the connector falls behind, it uses a larger batch size.  These batch configurations are supplied using the "log.mining.batch.size.min", "log.mining.batch.size.max", and "log.mining.batch.size.default" settings.  The message you are referring to are related to these "log.mining.batch.size.*" configurations and not the "max.batch.size".

That said, forcing LogMiner to read a batch of 200M is really not recommended. 

It's important to remember that a single SCN can represent up to a maximum of 96k changes since an SCN is not unique.  So in a worse case scenario, a 200M batch size can yield a result-set of 19.3 trillion logical changes, which I suspect is not what you want nor the hardware to support such a result set cached in the SGA.  Out of the box, the connector starts with a batch size of 10k and is allowed to grow to 100k, so at the worse case that could represent 9.6 billion logical changes and the smallest case, only 100k changes per batch. 

So first off, make sure you're setting the correct configurations as I don't believe what you've set is correct.

And secondly, be aware of the potential scaling that can happen per SCN depending on your system's load and transaction volume.

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/a6707b1d-fbd1-4e4f-8326-d19939c2a795n%40googlegroups.com.

Lucas Rangel Gazire

unread,
Jul 18, 2024, 1:55:01 PM7/18/24
to debezium
Hi Chris,
These are the values all right, i know it's wrong but i can't understand why setting a high value the messages keep showing. Is there a solution to avoid this?
I set it back to default value.

.with("log.mining.batch.size.min", minBatchSize.toBigDecimal().toLong())
.with("log.mining.batch.size.max", maxBatchSize.toBigDecimal().toLong())
.with("log.mining.batch.size.default", defaultBatchSize.toBigDecimal().toLong())

Chris Cranford

unread,
Jul 19, 2024, 7:43:08 AM7/19/24
to debe...@googlegroups.com
Hi Lucas -

So this specific message happens when the currently calculated mining end range is less-than or equal-to the previous mining end range and that the connector has fallen behind.  In short, it means despite the large batch range you've specified, the connector has fallen so far behind, it's trying to catch up by selecting the CURRENT_SCN as the upper bounds in favor of consuming SGA and Heap in a last ditch effort to catch up.  However, what it does not appear this specific log message takes into account is the configured max batch size.  There have been some small tweaks from contributors here to this method over the last 12 months or so, perhaps a subtle bug was introduced, can you raise a Jira?

Out of curiosity, what exactly is your MillisecondsBehindSource metric that represents the current lag? 

Thanks,
-cc


Lucas Rangel Gazire

unread,
Jul 19, 2024, 8:03:15 AM7/19/24
to debezium
Sure, i will create a Jira card, is there any specific format for the text?

>>Out of curiosity, what exactly is your MillisecondsBehindSource metric that represents the current lag? 
9.18minutes (550809ms)

Chris Cranford

unread,
Jul 19, 2024, 8:18:50 AM7/19/24
to debe...@googlegroups.com
Hi Lucas -

Wow, if you're only 9m with latency and those batch sizes, is the transaction volume on this database that massive?

Chris

Lucas Rangel Gazire

unread,
Jul 19, 2024, 8:29:29 AM7/19/24
to debezium
Yes, we have a massive volume transaction, it's our biggest issue today manage these data in almost real time.

Chris Cranford

unread,
Jul 19, 2024, 9:22:53 AM7/19/24
to debe...@googlegroups.com
Hi Lucas -

In case Jira is replying slowly via email, I am curious what are your JMX metrics for "OffsetScn", "CurrentScn", and "BatchSize" when you have this type of latency.

Thanks,
-cc

Lucas Rangel Gazire

unread,
Jul 19, 2024, 9:27:31 AM7/19/24
to debezium
I'll update right now
Reply all
Reply to author
Forward
0 new messages