Debezium Oracle source - large source lag

239 views
Skip to first unread message

Michael Slavev

unread,
Oct 18, 2024, 12:36:11 AMOct 18
to debezium

Hello,

I hope you're all doing well. We're working on real-time replication in production from several databases, but we're encountering significant lag from one particular source. The lag typically ranges from 5 to 10 minutes, occasionally reaching up to 200 minutes, depending on the load.

Our cluster is dedicated to Debezium replication and consists of three broker nodes, each running Kafka Broker, Kafka Connect, and Schema Registry. Each node has 8 vCPUs and 16 GB of RAM (with 6 GB heap to broker and 4 GB heap to Connect). We're running Confluent Kafka/Connect/Schema Registry 7.5 with Zulu JDK 11 and using the Debezium Oracle Connector version 2.5.2.

The problematic connector handles updates on 75 tables, with an average of 5 million events per day, peaking around midday when the lag is at its worst. I’ve tried increasing the heap for Kafka Connect, but it wasnt fully utilized. I also split the tables across three connectors, but this hasn’t improved performance.

After monitoring the production load with JMX, I noticed that QueueRemainingCapacity rarely drops below 7,000, while QueueTotalCapacity is set to the default of 8,192.

Any insights or suggestions would be appreciated.

Thank you!


Current source config:
{
    "config": {
        "connector.class": "io.debezium.connector.oracle.OracleConnector",
        "database.dbname": "xxx",
        "database.encrypt": "false",
        "database.hostname": "xxx",
        "database.password": "xxx",
        "database.pdb.name": "xxx",
        "database.port": "xxx",
        "database.user": "c##dbz",
        "decimal.handling.mode": "double",
        "errors.max.retries": "10",
        "field.name.adjustment.mode": "avro",
        "heartbeat.action.query": "UPDATE DEBEZIUM_HEARTBEAT SET ts=CURRENT_TIMESTAMP WHERE id = 'TIA_PROD'",
        "heartbeat.interval.ms": 60000,
        "log.mining.batch.size.default": 50000,
        "log.mining.batch.size.min": 10000,
        "log.mining.strategy": "online_catalog",
        "max.batch.size": 2048,
        "max.queue.size": 8192,
        "retriable.restart.connector.wait.ms": "900000",
        "schema.history.internal.consumer.sasl.jaas.config": "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"xxx\" password=\"xxx\";",
        "schema.history.internal.consumer.sasl.mechanism": "PLAIN",
        "schema.history.internal.consumer.security.protocol": "SASL_SSL",
        "schema.history.internal.consumer.ssl.truststore.location": "/var/ssl/private/kafka.client.truststore.jks",
        "schema.history.internal.consumer.ssl.truststore.password": "xxx",
        "schema.history.internal.kafka.bootstrap.servers": "xxx:9093,xxx:9093,xxx:9093",
        "schema.history.internal.kafka.topic": "debezium.TIA_PROD.schemaChanges",
        "schema.history.internal.producer.sasl.jaas.config": "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"xxx\" password=\"xxx\";",
        "schema.history.internal.producer.sasl.mechanism": "PLAIN",
        "schema.history.internal.producer.security.protocol": "SASL_SSL",
        "schema.history.internal.producer.ssl.truststore.location": "/var/ssl/private/kafka.client.truststore.jks",
        "schema.history.internal.producer.ssl.truststore.password": "xxx",
        "schema.history.internal.store.only.captured.databases.ddl": true,
        "schema.history.internal.store.only.captured.tables.ddl": true,
        "snapshot.mode": "schema_only",
        "table.include.list": "<list of 75 tables>",
        "tasks.max": "1",
        "time.precision.mode": "connect",
        "tombstones.on.delete": "true",
        "topic.prefix": "debezium.TIA_PROD"
    },
    "name": "TIA_PROD_debezium_source_2024-10-17_14-54-19"
}

Screenshot of jmx metrics for that source connector:
aa.png

jiri.p...@gmail.com

unread,
Oct 18, 2024, 12:37:30 AMOct 18
to debezium
Hi,

is it possible that the problematic connector needs to process very large transactions?

Jiri

Michael Slavev

unread,
Oct 18, 2024, 4:36:43 AMOct 18
to debezium
It depends on what is considered large.
We have about 3-5 tables, each with around 50 columns, and the values are typically up to 1 KB in size (with no LOBs involved).

Additionally, when I check the source database's V$LOG, there is only one current group, though I'm unsure if this could be contributing to the issue.
a.png

Dne pátek 18. října 2024 v 6:37:30 UTC+2 uživatel jiri.p...@gmail.com napsal:

Michael Slavev

unread,
Oct 22, 2024, 8:58:33 AMOct 22
to debezium
Hi,

Our DBA shared this screenshot with me.
a.png

Just before that, I took a screenshot showing the throughput and lag.
If I’m interpreting this correctly, our throughput is steady and slightly below 250 transactions per second.
a.png


Thank you,
Michael

Dne pátek 18. října 2024 v 6:37:30 UTC+2 uživatel jiri.p...@gmail.com napsal:
Hi,

Chris Cranford

unread,
Oct 23, 2024, 9:03:43 AMOct 23
to debe...@googlegroups.com
Hi Michael -

Can you run this query and share the results:

    SELECT INST_ID, GROUP#, THREAD#, BYTES
    from GV$LOG;

This will let us know what your redo log group sizes are configured as.

You also mentioned that "table.include.list" has 75 different tables. Do you have a number of other tables in this instance that are modified outside those 75? If so, have you considered trying "log.mining.query.filter.mode" set to "in"?

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 on the web visit https://groups.google.com/d/msgid/debezium/45ec1df9-7a8e-41da-b33a-5eeccb08af3fn%40googlegroups.com.

Michael Slavev

unread,
Oct 23, 2024, 10:28:18 AMOct 23
to debezium
Hi Chris,

I've attached the full query result bellow:
a.png

Yes, we have 75 different tables for which we had to work with the DBA to enable supplemental logging. Additionally, there are over 3,600 tables in this DB instance, and I believe they are all logging changes into the Redo logs, which have a 48-hour retention period. 

I haven't tried using "log.mining.query.filter.mode" set to "in", because our current mining strategy is set to "online_catalog" and initially, I overlooked it, thinking it only affected Redo log mining. However, I’ll give it a try after updating the connector to version 2.7.3.Final today, as I can’t upgrade Java to a newer JDK right now.

Thanks,
Michael


Dne středa 23. října 2024 v 15:03:43 UTC+2 uživatel Chris Cranford napsal:

Chris Cranford

unread,
Oct 23, 2024, 1:43:17 PMOct 23
to debe...@googlegroups.com
Hi Michael -

Thanks for the information. Let's see if the filter mode helps, that should reduce the bandwidth of data being sent between the database & the connector, particularly for those other ~3500 tables that are logging changes but aren't of importance.

As for the redo logs, this is a bit trickier. :(

Oracle DBAs often size redo logs based on a multitude of factors related to your Oracle instance, including recovery expectations. When redo logs are sized larger, this will cause more latency while the redo logs are read/parsed. This is due to the fact that Oracle LogMiner must read the entire log before data can be returned to Debezium. This means that if a redo log holds changes for SCNs between 1 to 1,000,000 and the connector is only interested in changes between SCNs 990,000 and 1,000,000, all changes in that log are read and many discarded by LogMiner automatically. So when using the `online_catalog` mode, the smaller the redo log sizes, the faster performance users often experience. But as I said, it's a balancing act because you have to weigh that against the other factors that DBAs use to size redo logs for the instance.

If I recall correctly, we had a community user in Brazil who had their redo log sizes around 6GB and they reduced them to 4GB and noticed a significant improvement. They also implemented the "log.mining.query.filter.mode" using the "in" strategy locally before it was a Debezium feature, and that also helped their performance, too. So I would recommend trying with the query filter mode first, measure the latency and if that doesn't help sufficiently, perhaps the DBA could revisit the log sizes?

Thanks,
-cc

Michael Slavev

unread,
Oct 24, 2024, 8:44:04 AMOct 24
to debezium

Hi Chris,

I've made several iterations with the filter mode enabled and adjusted parameters, and so far, it has significantly improved performance. However, a few issues remain.

  1. Log Mining Batch Size: I experimented with the log.mining.batch.size setting, as the default values were causing bottlenecks—batches were being exhausted at 100k, even after enabling filter mode.

  2. Producer Configuration: I also updated the Kafka Connect producer settings (batch size, max request size, linger, and compression). These settings were previously set to the default Confluent values, which seemed to cause a bottleneck between Connect and the broker.

  3. Batch and Queue Sizes: Finally, I tweaked the max.batch.size and max.queue.size settings, which are now slightly overestimated and not fully utilized.

Here’s the configuration I'm working with:
{

    "config": {
        "connector.class": "io.debezium.connector.oracle.OracleConnector",
        "database.dbname": "xxx",
        "database.encrypt": "false",
        "database.hostname": "xxx",
        "database.password": "xxx",
        "database.pdb.name": "xxx",
        "database.port": "xxx",
        "database.user": "c##dbz",
        "decimal.handling.mode": "double",
        "errors.max.retries": "10",
        "field.name.adjustment.mode": "avro",
        "heartbeat.action.query": "UPDATE DEBEZIUM_HEARTBEAT SET ts=CURRENT_TIMESTAMP WHERE id = 'TIA_PROD'",
        "heartbeat.interval.ms": 60000,
        "log.mining.batch.size.default": 50000,
        "log.mining.batch.size.max": 500000,
        "log.mining.batch.size.min": 50000,
        "log.mining.query.filter.mode": "in",
        "log.mining.strategy": "online_catalog",
        "max.batch.size": 262144,
        "max.queue.size": 1048576,
        "poll.interval.ms": "200",
        "producer.override.batch.size": "524288",
        "producer.override.compression.type": "lz4",
        "producer.override.linger.ms": "100",
        "producer.override.max.request.size": "10485760",

        "retriable.restart.connector.wait.ms": "900000",
        "schema.history.internal.consumer.sasl.jaas.config": "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"xxx\" password=\"xxx\";",
        "schema.history.internal.consumer.sasl.mechanism": "PLAIN",
        "schema.history.internal.consumer.security.protocol": "SASL_SSL",
        "schema.history.internal.consumer.ssl.truststore.location": "/var/ssl/private/kafka.client.truststore.jks",
        "schema.history.internal.consumer.ssl.truststore.password": "xxx",
        "schema.history.internal.kafka.bootstrap.servers": "czsrepkafkap1.cpas.cz:9093,czsrepkafkap2.cpas.cz:9093,czsrepkafkap3.cpas.cz:9093",

        "schema.history.internal.kafka.topic": "debezium.TIA_PROD.schemaChanges",
        "schema.history.internal.producer.sasl.jaas.config": "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"xxx\" password=\"xxx\";",
        "schema.history.internal.producer.sasl.mechanism": "PLAIN",
        "schema.history.internal.producer.security.protocol": "SASL_SSL",
        "schema.history.internal.producer.ssl.truststore.location": "/var/ssl/private/kafka.client.truststore.jks",
        "schema.history.internal.producer.ssl.truststore.password": "xxx",
        "schema.history.internal.store.only.captured.databases.ddl": true,
        "schema.history.internal.store.only.captured.tables.ddl": true,
        "snapshot.mode": "schema_only",
        "table.include.list": "<list of 75 tables>",
        "tasks.max": "1",
        "time.precision.mode": "connect",
        "tombstones.on.delete": "true",
        "topic.prefix": "debezium.TIA_PROD"
    },
    "name": "TIA_PROD_debezium_source_2024-10-24_14-19-56"
}

The configuration did provide some positive results initially, with lag staying within reasonable limits for hour or so. However, after a batch size spike, the throughput dropped slightly, though it's still more than double what it was before.
Interestingly, QueueRemainingCapacity remains mostly unused.


a.png

Let me know your thoughts or if you have any other suggestions for improvements — aside from adjusting the log size with the DBA, because it's a core database, any such change is rarely approved.

Thank you!
Michael
Dne středa 23. října 2024 v 19:43:17 UTC+2 uživatel Chris Cranford napsal:

Chris Cranford

unread,
Oct 25, 2024, 1:13:24 AMOct 25
to debe...@googlegroups.com
Hi Michael -

So the only point I'd like to caution with is the log.mining.batch.size.* settings.

What users are often not aware of is that Oracle can assign up to 16k unique changes (redo entries) to a single Oracle system change number on Oracle 12 and up to 96k unique changes (redo entries) on Oracle 19 or later. When you configure the log.mining.batch.size.*, this isn't referring to the number of unique changes in the batch, but rather the number of unique system change numbers, so a worse case scenario is that a max batch size of 500k can yield a result set of 48M individual redo entry / changes per result set. That could easily exhaust the JVM heap depending on your column types and their data. It's important to fully understand at your database's peak load what your transaction volume and the number of unique changes are being assigned to the same SCN to avoid reaching an OutOfMemoryError.

In most normal situations where your database creates very small lived transactions, this is a non-issue. Where this often can be problematic is if you're doing data loads, in such cases its quite common for Oracle to assign hundreds, if not thousands of inserts or updates to the same SCN.

Otherwise, I'm glad you're seeing better performance \o/

Thanks,
-cc

Michael Slavev

unread,
Nov 1, 2024, 9:32:34 AMNov 1
to debezium
Hi Chris,

Since our DBA is currently on a 14-day vacation, there’s no update yet on the redo log size.

Since my last update, the source lag appeared to improve, with a maximum of around 15 minutes throughout the day. However, during peak usage today, it spiked back up to over 20 minutes a few times (see attached screenshot).

To get a clearer view of the issue, I’ve set up Prometheus and Grafana for monitoring, but so far I haven’t identified anything conclusive. I’ve also reduced the previously oversized max.batch.size and max.queue.size parameters, which were rarely being fully utilized.
Additionally, I lowered log.mining.batch.size based on your recommendation, which seems to hit its maximum during periods of higher load, even if it’s 500k or 200k.

With this improved monitoring setup, I noticed that the SCN gap has been growing significantly today — previously reaching up to 30 million before I deployed the connector with updated configurations.
Despite the changes, the SCN gap is still increasing over time (currently 2 milion).

Is there anything I might be overlooking, or any additional adjustments you would recommend?

Current config:

{
    "connector.class": "io.debezium.connector.oracle.OracleConnector",
    "database.dbname": "xxx",
    "database.encrypt": "false",
    "database.hostname": "xxx",
    "database.password": "xxx",
    "database.pdb.name": "xxx",
    "database.port": "1521",

    "database.user": "c##dbz",
    "decimal.handling.mode": "double",
    "errors.max.retries": "10",
    "field.name.adjustment.mode": "avro",
    "heartbeat.action.query": "UPDATE DEBEZIUM_HEARTBEAT SET ts=CURRENT_TIMESTAMP WHERE id = 'TIA_PROD'",
    "heartbeat.interval.ms": 60000,
    "log.mining.batch.size.default": 50000,
    "log.mining.batch.size.max": 200000,
    "log.mining.batch.size.min": 25000,

    "log.mining.query.filter.mode": "in",
    "log.mining.strategy": "online_catalog",
    "max.batch.size": 8192,
    "max.queue.size": 32768,
    "poll.interval.ms": "200",
    "producer.override.batch.size": "1048576",

    "producer.override.compression.type": "lz4",
    "producer.override.linger.ms": "100",
    "producer.override.max.request.size": "10485760",
    "retriable.restart.connector.wait.ms": "900000",
    "schema.history.internal.consumer.sasl.jaas.config": "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"xxx\" password=\"xxx\";",
    "schema.history.internal.consumer.sasl.mechanism": "PLAIN",
    "schema.history.internal.consumer.security.protocol": "SASL_SSL",
    "schema.history.internal.consumer.ssl.truststore.location": "/var/ssl/private/kafka.client.truststore.jks",
    "schema.history.internal.consumer.ssl.truststore.password": "xxx",
    "schema.history.internal.kafka.bootstrap.servers": "xxx:9093,xxx:9093,xxx:9093",
    "schema.history.internal.kafka.topic": "debezium.TIA_PROD.schemaChanges",
    "schema.history.internal.producer.sasl.jaas.config": "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"xxx\" password=\"xxx\";",
    "schema.history.internal.producer.sasl.mechanism": "PLAIN",
    "schema.history.internal.producer.security.protocol": "SASL_SSL",
    "schema.history.internal.producer.ssl.truststore.location": "/var/ssl/private/kafka.client.truststore.jks",
    "schema.history.internal.producer.ssl.truststore.password": "xxx",
    "schema.history.internal.store.only.captured.databases.ddl": true,
    "schema.history.internal.store.only.captured.tables.ddl": true,
    "snapshot.mode": "schema_only",
    "table.include.list": "<75 tables>",

    "tasks.max": "1",
    "time.precision.mode": "connect",
    "tombstones.on.delete": "true",
    "topic.prefix": "debezium.TIA_PROD"
}

Todays metrics:
a.png


Thank you so much for you time!
Michael
Dne pátek 25. října 2024 v 7:13:24 UTC+2 uživatel Chris Cranford napsal:

Michael Slavev

unread,
Nov 20, 2024, 7:44:04 AMNov 20
to debezium

Hi Chris,

I wanted to share a quick update after our last meeting. Upon reviewing the Oracle Alert Logs, I’ve identified a possible culprit for the periodic source lag. The issue seems to occur after every second or third log switch. I’ve attached two alert logs from yesterday for your reference—one shows a seamless log switch, while the other (15 minutes later) introduced an 8 minute lag.

We haven’t been able to tie this behavior to CPU or disk I/O usage on the database. As we discussed earlier, the database is running on ASM with optical fiber and enterprise-grade specialized disks. The bandwidth between the database and Debezium doesn’t appear to be a suspect either.

Additionally, there have been a few recent changes to the database log setup:

  • Log group count: Increased from 3 to 4.
  • Log size: Decreased from 5.1GB to 4GB.

Despite these changes, there hasn’t been any noticeable improvement in lag. This might be due to the fact that the log switches typically occur when the logs reach around 2.5GB during working hours, rather than utilizing the full log size.

If you have any further recommendations or areas we should review with the DBA, please let me know.

Thank you,
Michael

Dne pátek 1. listopadu 2024 v 14:32:34 UTC+1 uživatel Michael Slavev napsal:
log_switch-no_lag.log
log_switch_lag.log

Chris Cranford

unread,
Nov 20, 2024, 9:09:48 AMNov 20
to debe...@googlegroups.com, Michael Slavev
Hi Michael -

The only thing the alert logs show is that in the first switch, the delta between the lower and upper SCN bounds just before and after the switch seem to range just under 40k. In the case of where you observed the latency, the variance is closer to 300k.

Do you believe your DBA would be willing to run an ad-hoc LogMiner extract for me? If so, the data extract I'd like to get would be:

    SELECT SCN, RAWTOHEX(XID), OPERATION, OPERATION_CODE, TIMESTAMP, SEG_OWNER, TABLE_NAME, ROW_ID, CSF, ROLLBACK_FLAG
    FROM V$LOGMNR_CONTENTS
    WHERE SCN > ? AND SCN <= ?

This query doesn't provide me the SQL so there shouldn't be any sensitive data here, but if you'd prefer to send that privately, feel free to DM me on Zulip or send it to my email directly.

The SCN ranges I am interested in specifically are as follows:

    9763292654374 - 9763292707234 (no lag window)
    9763293538356 - 9763294078347 (lag window)

What I am interested in seeing in these two extracts is the number of concurrent transactions in the window, but more importantly what are the operations involved and whether any of them specifically specify ROLLBACK_FLAG with a value of 1, which we have recently identified as a performance bottleneck.

Thanks,
-cc

Michael Slavev

unread,
Nov 20, 2024, 10:36:43 AMNov 20
to debezium

Hi Chris,

Thank you for getting back to me so quickly. I’ve compiled an Excel with two lists: one containing the log mining results for the no-lag scenario, and another with results from when the lag was significant.

Due to file size limitations on Google Groups, I’ve uploaded the zip file to Google Drive for easier access - https://drive.google.com/file/d/13sewTpcQ3AXDwdyQNaauEZMQDlh9bN7c/view?usp=sharing

If it's needed, tables we query are following: "table.include.list": "TIA.ACC_ITEM,TIA.ACC_PAYMENT_DETAILS,TIA.AGREEMENT_LINE,TIA.CASE_ITEM,TIA.CASE_ITEM_ATTACHMENTS,TIA.CLA_CASE,TIA.CLA_CODE,TIA.CLA_CUBE,TIA.CLA_EVENT,TIA.CLA_EVENT_LOG,TIA.CLA_ITEM,TIA.CLA_PAYMENT_ITEM,TIA.CLA_QUESTION,TIA.CLA_SUBCASE,TIA.CLA_THIRD_PARTY,TIA.HISTORY_LOG,TIA.INTERESTED_PARTY_IN_OBJ,TIA.NAME,TIA.NAME_TELEPHONE,TIA.TCP_CLA_E_COMMUNICATION,TIA.OBJECT,TIA.OBJ_RISK_SLAVE,TIA.POLICY,TIA.POLICY_ENTITY,TIA.PRINT_ARGUMENT,TIA.PRINT_REQUEST,TIA.PRODUCT_LINE,TIA.RELATION,TIA.TARIFF_CODES,TIA.TARIFF_RATING,TIA.TARIFF_STRUCTURE,TIA.TCP_ANU_CLAIM_RESERVE,TIA.TCP_ANU_CNU,TIA.TCP_ANU_MODEL_VOUCHER,TIA.TCP_CLAIM_CONTACT,TIA.TCP_CLAIM_NAME_ROLES,TIA.TCP_CLA_ACCESS_AUDIT,TIA.TCP_CLA_COINSURANCE,TIA.TCP_CLA_COINSURANCE_SHARES,TIA.TCP_CLA_CRS_COMMUNICATION,TIA.TCP_CLA_CRS_DATA,TIA.TCP_CL_RECEIVABLES_OUT,TIA.TCP_DELEGATE_USER,TIA.TCP_EMAIL_REQUEST,TIA.TCP_HOME_SERVICE,TIA.TCP_INET_NONTASK_INSP,TIA.TCP_INET_STAT_PLACE,TIA.TCP_KCC_H_VAZBALPRIZIKOTIA,TIA.TCP_KCC_RISK_XREF,TIA.TCP_PL_PLACE_OF_INSURANCE,TIA.TCP_POLICY_NAME_ROLES,TIA.TCP_POL_AGENT_COMMISSION_NO,TIA.TCP_THIRD_PARTY_POV,TIA.TCP_USER_QUALIF_FA,TIA.TCP_VEHICLE_CARD,TIA.TIA_USER_PROFILE,TIA.TOP_USER,TIA.WORK_GROUP_MEMBER,TIA.TCP_CLA_COINSURANCE_REI,TIA.TCP_QUALIFICATION,TIA.TCP_DOCS_EXPECTED,TIA.TCP_POLICY_CANCELLATION,TIA.TCP_KCC_H_VAZBABALICEKLP,TIA.TCP_USER_AGENT,TIA.POL_REFERRAL,TIA.TCP_GIS_DETAILS,TIA.TCP_B1_GIS_DETAILS,TIA.POST_CODE,TIA.TCP_EMAIL_ARGUMENT,TIA.TCP_PROD_COV_ADDRESS,TIA.TCP_SEG_VIP,TIA.XLA_PE_REFERENCE,TIA.ACC_ITEM_LOG,TIA.TCP_QUALIFICATION_HIST,TIA.TCP_USER_QUALIF_FA_HIST"

Let me know if there’s anything else you need!

Thank you,
Michael




Dne středa 20. listopadu 2024 v 15:09:48 UTC+1 uživatel Chris Cranford napsal:

Michael Slavev

unread,
Dec 4, 2024, 11:22:35 AMDec 4
to debezium

Hi Chris,

Just checking in—any updates regarding the suspicious log switching and rollbacks?

Thanks,
Michael

Dne středa 20. listopadu 2024 v 16:36:43 UTC+1 uživatel Michael Slavev napsal:

Chris Cranford

unread,
Dec 18, 2024, 4:47:59 AM (9 days ago) Dec 18
to debe...@googlegroups.com
Hi Michael

The only thing I can share right now is we have determined that partial roll backs are an issue and when they're done in high number, this can lead to performance degradation.  In Debezium 3.0.5, there is a new metric that tracks partial rollbacks, and that should help determine the severity of that action on connector deployments and the windows where that happens, which can will help highlight reasons for lag during specific time periods.  As to how to address the partial rollbacks, we're looking into options, and worse case we may need to revert some internal refactorings that took place in recent builds.  I'll share more as I have more details.

-cc
Reply all
Reply to author
Forward
0 new messages