[Debezium Oracle Connector] Let me ask you some questions.

1,669 views
Skip to first unread message

masami310

unread,
Mar 2, 2022, 7:57:11 AM3/2/22
to debezium
Hi,

There are a few things I'm curious about in the production Debezium logs, let me ask you a question.

1. The log "The connector is now using the maximum batch size 100000 when querying the LogMiner view. This could be directive of large SCN gaps" is frequently output.
How much should this increase the value of log.mining.batch.size.max?
I would like advice for determining the setting value.

2. When is "Streaming metrics dump:" output to the log?

3. What period of lag does LagFromTheSourceDuration measure?
For example, is it the period from the date and time when the change data was written to the archive log to the date and time when the message was sent to the Kafka broker?

4. Currently, there is a big lag in the production environment.
I would like advice on possible causes and countermeasures.
maxLagFromTheSourceDuration = PT12H57M58.910002S

--- Streaming metrics dump (all)---
2022-02-28T19:05:30.657Z [2022-03-01 04:05:30,632] INFO [bakskvs-prod-connector|task-0] Streaming metrics dump:
OracleStreamingChangeEventSourceMetrics{
  currentScn=744984117800, oldestScn=744289067821, committedScn=744846515060, offsetScn=744289067820,
  logMinerQueryCount=8332, totalProcessedRows=204076234, totalCapturedDmlCount=36874626,
  totalDurationOfFetchingQuery=PT33H25M18.172616S, lastCapturedDmlCount=40,
  lastDurationOfFetchingQuery=PT15.264643S, maxCapturedDmlCount=9192797,
  maxDurationOfFetchingQuery=PT6M10.491418S, totalBatchProcessingDuration=PT78H56M59.640099S,
  lastBatchProcessingDuration=PT29.122857S, maxBatchProcessingThroughput=10500,
  currentLogFileName=[+DATAC1/BXXXX11/ONLINELOG/group_3.1816.1069802111,
                      +DATAC1/BXXXX11/ONLINELOG/group_11.1125.1069802131],
  minLogFilesMined=2, maxLogFilesMined=2,
  redoLogStatus=[+DATAC1/BXXXX11/ONLINELOG/group_1.1780.1069802107
    | ACTIVE, +DATAC1/BXXXX11/ONLINELOG/group_4.1817.1069802115
    | ACTIVE, +DATAC1/BXXXX11/ONLINELOG/group_2.1813.1069802109
    | ACTIVE, +DATAC1/BXXXX11/ONLINELOG/group_10.1475.1069802127
    | ACTIVE, +DATAC1/BXXXX11/ONLINELOG/group_11.1125.1069802131
    | CURRENT, +DATAC1/BXXXX11/ONLINELOG/group_3.1816.1069802111
    | CURRENT, +DATAC1/BXXXX11/ONLINELOG/group_9.1822.1069802125
    | INACTIVE, +DATAC1/BXXXX11/ONLINELOG/group_12.1124.1069802133
    | INACTIVE],
  switchCounter=223, batchSize=100000, millisecondToSleepBetweenMiningQuery=0,
  hoursToKeepTransaction=0, networkConnectionProblemsCounter0, batchSizeDefault=20000,
  batchSizeMin=1000, batchSizeMax=100000, sleepTimeDefault=1000, sleepTimeMin=0,
  sleepTimeMax=3000, sleepTimeIncrement=200, totalParseTime=PT6M28.215043S,
  totalStartLogMiningSessionDuration=PT1M20.994204S, lastStartLogMiningSessionDuration=PT0.02101S,
  maxStartLogMiningSessionDuration=PT0.370744S, totalProcessTime=PT78H56M59.640099S,
  minBatchProcessTime=PT0.071576S, maxBatchProcessTime=PT4H59M32.704482S,
  totalResultSetNextTime=PT42H1M54.906585S, lagFromTheSource=DurationPT8H51M18.214495S,
  maxLagFromTheSourceDuration=PT12H57M58.910002S, minLagFromTheSourceDuration=PT0.427924S,
  lastCommitDuration=PT0.000004S, maxCommitDuration=PT27M1.130142S, activeTransactions=56,
  rolledBackTransactions=524260, committedTransactions=83045739, abandonedTransactionIds=[],
  rolledbackTransactionIds=[2e001b00f8730000, 37001700f49c0000, …, 2a002100ef670200],
 registeredDmlCount=36875340, committedDmlCount=33985075, errorCount=1, warningCount=494,
 scnFreezeCount=0, unparsableDdlCount=494, miningSessionUserGlobalAreaMemory=48266632,
 miningSessionUserGlobalAreaMaxMemory=53287192, miningSessionProcessGlobalAreaMemory=114296296,
 miningSessionProcessGlobalAreaMaxMemory=346424808
}(io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource:195)

---
The version of Debezium is 1.8.1.Final.
Oracle is 12.1.
Kafka broker(Amazon MSK) 2.8.1.

Chris Cranford

unread,
Mar 2, 2022, 1:29:15 PM3/2/22
to debe...@googlegroups.com, masami310
Hi, see my responses inline...


On 3/2/22 07:57, masami310 wrote:
Hi,

There are a few things I'm curious about in the production Debezium logs, let me ask you a question.

1. The log "The connector is now using the maximum batch size 100000 when querying the LogMiner view. This could be directive of large SCN gaps" is frequently output.
How much should this increase the value of log.mining.batch.size.max?
I would like advice for determining the setting value.
That depends entirely on the load of your database.  The maximum batch size has a direct impact on the amount of PGA memory space used by the connector's mining session, and going over the available PGA memory will lead to a PGA_AGGREGATE_LIMIT error and the connector will stop.  In general, 100k should be sufficient but you could try increasing it by 50k or a 100k and see if you experience any PGA memory issues.  Just also understand the larger the batch, the longer it will take for results to be collected and made available per mining iteration as well, so you may need to experiment with what works best. 

This output can occur under several conditions, including situations where a hot backup runs and advanced the SCN by a large window.  There is logic in place to detect this and advance the jump faster, so if you're seeing this message repeatedly, then chances are this isn't a product of an SCN jump creating a gap but more likely the fact you have a lot of changes in your environment that need to be mined.  This can happen due to bulk load or bulk changes or even if the connector was down for an extended period of time and its trying to catch up to real-time events.


2. When is "Streaming metrics dump:" output to the log?
Only when an exception occurs or when the connector is gracefully stopped.


3. What period of lag does LagFromTheSourceDuration measure?
For example, is it the period from the date and time when the change data was written to the archive log to the date and time when the message was sent to the Kafka broker?
Close, it's the difference between the time when the change occurred in the redo logs & when it was seen by Debezium.


4. Currently, there is a big lag in the production environment.
I would like advice on possible causes and countermeasures.
maxLagFromTheSourceDuration = PT12H57M58.910002S
This could be any one of or a combination of the following (this is not an exhaustive list):

    * Connector was down for an extended period of time and restarted
    * There was a bulk load/change of data that created a burst of redo entries
    * The entire database is configured to log all changes, creating lots of unnecessary noise in the logs that has to be filtered
    * Redo logs are configured poorly and lots of archive logs are being generated unnecessary, creating unnecessary IO to catch up
    * You have enabled LOB support and have a long-running transaction
    * There is a timezone bug in the lag calcuation
Looking at your metrics dump there are few things that raise concern.

The first is your SCN values for currentScn, oldestScn, committedScn, and offsetScn.  If we look at currentScn vs oldestScn, we'll see that you have 695,049,979 changes between where the database is current at and the oldest change in the buffer associated with the oldest in-progress transaction.  That's nearly 700 million changes, which is really very substantial.  Furthermore if we look at committedScn, this represents the most recent committed transaction we've emitted to Kafka.  If we compare that to oldestScn to this, that is 557,447,239 changes, which tells me you either have some very long-running transactions or you have a lot of bulk operations happening in your environment.   That is also pretty substantial and very unexpected. 

Secondly if we look at the duration values we'll see the lastDurationOfFetchingQuery took 15 seconds to get results from LogMiner.  Given your volume and 100,000 batch range, that's probably reasonable since logs must be read sequentially.  Each batch (or poll for records) from LogMiner is being processed at a rate of 10500 events/second, which isn't necessarily bad as there is a lot of overhead that can influence this.  For example, your maxBatchProcessTime is nearly 5 hours which is insane.  Did you have an unusually large transaction?  What makes me think you did is maxCommitDuration which is just over 27 minutes.  This is the time it took the connector to package up a transaction and send it to Kafka.  During that 27 minute window, the connector is unable to read anymore records from LogMiner, perpetually falling behind.  If you have many occurrences of this, that might explain the lag.

Thirdly, active transactions reports 56.  That by itself isn't necessarily a cause for concern across multiple log mining sessions if those transactions are short-lived; however, if they're lasting for more than a few seconds, this exacerbates the buffering and dispatch of events to Kafka.  Unlike other databases, Oracle writes everything to the redo logs, including things that are eventually rolled back as well as lots of internal database changes that we aren't concerned about from a CDC perspective.  But all this must be read & excluded by the connector.  Since things are written to the logs as they happen (i.e. you start a transaction, perform the insert and let the session sit idle for 1 hour without a commit), that transaction gets written to the logs as soon as you start it.  The connector will then buffer that transaction in-memory (or disk if you use Infinispan buffering) until your session times out & Oracle rolls back the transaction or you eventually commit the transaction.  If that transaction lingers for hours or days, then the oldestScn in the metrics will never be able to advance forward and can be detrimental if you have LOB support enabled due to the nature of how LOB columns are mined.

Fourthly, what I do find most alarming is the totalResultSetNextTime, being over 42 hours.  I guess this ultimately would depend on the duration of the connector's uptime for that execution, but if a vast majority of the time is being spent in result-set next calls, this could be product of a poorly performing query against LogMiner so what we need to do is to look at your connector configuration.  Are you running one connector or multiple connectors?  What is your connector configuration, and please do not omit things such as table.include.list or schema.include.list values.  You can mask the names if you want but I need to know the number of tables or schemas in those filter configurations since they're used directly as part of the Oracle LogMiner query.

---
The version of Debezium is 1.8.1.Final.
Oracle is 12.1.
Kafka broker(Amazon MSK) 2.8.1.
--
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/9cc5e22a-bca0-4435-97de-39dd62ff5f21n%40googlegroups.com.

masami310

unread,
Mar 3, 2022, 10:04:38 PM3/3/22
to debezium
Hi,

Thank you for all the useful advice.

There are only two tables to be captured, and they are set in table.include.list.
In addition, we understand that the two tables store message information associated with users, and that a large amount of data is created by batch jobs even during the daytime.

Let me ask you some additional questions.

> * Redo logs are configured poorly and lots of archive logs are being generated unnecessary, creating unnecessary IO to catch up
 It says "Redo logs are configured poorly", but what kind of settings are missing?

>* You have enabled LOB support and have a long-running transaction
lob.enabled = false (default value) and has not been changed.

>this could be product of a poorly performing query against LogMiner so what we need to do is to look at your connector configuration. 
The connector configuration is set as follows.
{
    "name": "kvs-prod-connector",
    "config": {
        "connector.class": "io.debezium.connector.oracle.OracleConnector",
        "tasks.max": "1",
        "database.server.name": "server1",
        "database.oracle.version": "12+",
        "database.user": "user",
        "database.password": "pw",
        "database.url": "url",
        "database.dbname": "xx",
        "database.pdb.name": "pdb",
        "database.connection.adapter": "logminer",
        "table.include.list": "SCHEMANAME.TABLE001,SCHEMANAME.TABLE003",
        "database.history.kafka.bootstrap.servers": "broker1:9092,broker2:9092",
        "database.history.kafka.topic": "data-changes.xx-prod",
        "snapshot.mode": "schema_only",
        "task.shutdown.graceful.timeout.ms": "20000",
        "log.mining.strategy": "online_catalog",
        "database.history.skip.unparseable.ddl": "true",
        "offset.flush.interval.ms": 10000,
        "offset.flush.timeout.ms": 60000,
        "max.batch.size": 1048,
        "max.queue.size": 4096
    }
}

>Are you running one connector or multiple connectors?  What is your connector configuration,
> and please do not omit things such as table.include.list or schema.include.list values.  
>You can mask the names if you want but I need to know the number of tables or schemas in those filter configurations since they're used directly as part of the Oracle LogMiner query.
There is only one connector.
table.include.list is set. (Specify 2 tables)
schema.include.list has received the advice of ↓ before, so now I am temporarily removing the setting.


2022年3月3日木曜日 3:29:15 UTC+9 Chris Cranford:

Chris Cranford

unread,
Mar 4, 2022, 12:55:27 PM3/4/22
to debe...@googlegroups.com, masami310
HI, see my responses inline...

On 3/3/22 22:04, masami310 wrote:
Hi,

Thank you for all the useful advice.

There are only two tables to be captured, and they are set in table.include.list.
In addition, we understand that the two tables store message information associated with users, and that a large amount of data is created by batch jobs even during the daytime.
Are these batch jobs creating large transactions that span many minutes or hours or are they doing smaller, chunk-based transactions?


Let me ask you some additional questions.

> * Redo logs are configured poorly and lots of archive logs are being generated unnecessary, creating unnecessary IO to catch up
 It says "Redo logs are configured poorly", but what kind of settings are missing?
You are using "online_catalog", so the size of your redo logs does not matter, this only applies if you're using "log.mining.strategy=catalog_in_redo".  In case others stumble onto this thread, I'm referring to the size of the log files being critical when using "catalog_in_redo".


>* You have enabled LOB support and have a long-running transaction
lob.enabled = false (default value) and has not been changed.
Ok, that simplifies things a bit.

masami310

unread,
Mar 7, 2022, 5:57:51 AM3/7/22
to debezium
Hi,


> Are these batch jobs creating large transactions that span many minutes or hours or are they doing smaller, chunk-based transactions?

It's a big transaction that lasts a few minutes.
Runs in the morning, noon and evening.

In the previous answer, it was written about "schema.include.list",
Should this parameter be set again for performance reasons?

If you have any other things to try with the connector configuration, please give us some advice.

There are two things we would like to try next.
1) Since you pointed out "maxCommitDuration = PT27M1.130142S", raise the specifications (instance type) of MSK (Kafka broker) by one level.
2) Limit the capture target table to only one (specify only one in table.include.list)

We want to realize the mechanism of the attached file (system_configuration.png).
I deployed this mechanism to a commercial environment in January of this year, but it is not stable and I am struggling.

Every time I have a problem, I ask this community a question, and it's always helpful to have a quick response.

My immediate goal is to get this mechanism up and running in a commercial environment.
And after stable operation, I would like to disseminate this achievement and knowledge in Japan.
I would like to ask you a lot of questions, but thank you for your continued support.

2022年3月5日土曜日 2:55:27 UTC+9 Chris Cranford:
system_configuration.png

Chris Cranford

unread,
Mar 7, 2022, 9:58:27 AM3/7/22
to debe...@googlegroups.com, masami310
Hi,

Can you describe how big this transaction is?  How many DML events happen in this transaction?  Could that transaction be broken up into small chunks perhaps?

CHris

masami310

unread,
Mar 7, 2022, 8:23:30 PM3/7/22
to debezium
Hi,

> Can you describe how big this transaction is?
> How many DML events happen in this transaction?

I don't know the details of the batch process because this transaction is managed outside of my department.
However, looking at the Oracle statistics, the number of occurrences per day is as follows.

INSERT: 5.3 million records
DELETE: 4.5 million records
UPDATE: 140,000 records

Is it impossible to use debezium with this transaction volume?


> Could that transaction be broken up into small chunks perhaps?

It's a big and complicated batch job, so it can't be easily repaired.
Refurbishment is expected to take a lot of time and money.


2022年3月7日月曜日 23:58:27 UTC+9 Chris Cranford:

Chris Cranford

unread,
Mar 11, 2022, 8:41:24 AM3/11/22
to debe...@googlegroups.com, masami310
Hi,

I wouldn't say that its impossible to use Debezium with this process, but it is important that the environment be configured to support the types of transactions you perform.  But it's also important to understand how these transactions play well or don't play well with Oracle LogMiner too.  Here are a few things I would suggest you tweak in your connector configuration to see if it helps at all given the volume of the batch job:

    - max.queue.size
        remove this, leave it as its default.  Ideally you would want this to be larger rather than small than defaults if changed.

    - max.batch.size
        remove this, leave it as its default.  Ideally you would want this to be larger rather than small than defaults if changed.

    - log.mining.batch.size.min
        The default is 1000, perhaps increase this to 10000

    - log.mining.batch.size.max
        The default is 100000, perhaps increase this to 200000

    - log.mining.batch.size.default
        The default is 20000, perhaps increase this to 50000.

    - log.mining.view.fetch.size
        The default is 10000, you may want to increase this to 20000 to read more rows per database fetch.

The log.mining.batch.size.[min|max|default] values are critical here.  The closer the connector moves toward the "max" means the more PGA memory the connector will require per mining iteration to load the data from the logs into memory.  If you run into PGA_AGGREGATE_LIMIT errors with the connector then either you will need to decrease these values or ask your DBA to increase the PGA memory pool so that you can mine larger batches.  The "log.mining.view.fetch.size" is how many records are read from the LogMiner view per database round-trip.  Ideally you want this to be a decent size based on how large your query result may be to limit the number of round trips; however its critical to remember that this also influences the memory consumption on the connector side.  The larger the fetch size, the more memory the connector will consume.  Lastly, since you're not familar with how the transaction(s) are built, you may want to look into using "log.mining.buffer.type=infinispan" and provide the "log.mining.buffer.infinispan.cache.*" properties.  The benefit of Infinispan caches is that they can be kept off-heap and may perform better than heap-based caches if the transactions are very large. 

Let us know how any of these help your performance
Chris

masami310

unread,
Mar 13, 2022, 9:25:26 PM3/13/22
to debezium
Hi,  Chris

Thank you for your useful advice
This week I'll change the configuration and give it a try.
The results will be reported at a later date.

2022年3月11日金曜日 22:41:24 UTC+9 Chris Cranford:
Reply all
Reply to author
Forward
0 new messages