Oracle signal is not working as expected

43 views
Skip to first unread message

Shahul Nagoorkani

unread,
Sep 25, 2025, 7:52:34 PM (6 days ago) Sep 25
to debezium
Hello folks,

We recently had an issue with the Postgres Signal issue which was resolved with the help of this community experts. We tried to follow the same method for Oracle as well except that Oracle doesn't need a publication where we add the signal table in case of postgres.

Here is the config:
kind: KafkaConnector
metadata:
name: debezium-connector-ebiz-walletload-big-tables
namespace: dba-debezium
labels:
strimzi.io/cluster: kafka-connect-dba-ebiz-wallet
spec:
class: io.debezium.connector.oracle.OracleConnector
tasksMax: 1
config:
tasks.max: 1
# Skip full load on first run, start on CDC. Use signal table to trigger incremental snapshots of big tables
snapshot.mode: configuration_based
snapshot.mode.configuration.based.snapshot.schema: true
snapshot.mode.configuration.based.snapshot.data: false
snapshot.mode.configuration.based.start.stream: true
incremental.snapshot.enabled: true
snapshot.max.threads: 3
# Incremental snapshot tuning
incremental.snapshot.chunk.size: 10000
snapshot.fetch.size: 20000

#Debezium Engine/Queing Settings
max.batch.size: 8192
max.queue.size: 65536
max.queue.size.in.bytes: 536870912 #512MB
# Schema refresh for DDL changes

#Oracle Driver Tuning
driver.defaultRowPrefetch: 60000
driver.oracle.net.SDU: 65535

poll.interval.ms: 20 # default 500; shorter limits idle time

log.mining.query.filter.mode: in

database.hostname: <hostname>
database.port: 1522
database.user: ${secrets:dba-debezium/ebiz-dba-wallet-secret:username}
database.password: ${secrets:dba-debezium/ebiz-dba-wallet-secret:password}
database.dbname: DB3L231
schema.include.list: WALLET,DEBEZIUM
table.include.list: WALLET.SYMADDRESS,WALLET.SYMCREDITCARD,WALLET.SYMCUREQUEST,WALLET.SYMDIRECTDEBIT,DEBEZIUM.DEBEZIUM_WAL_LOAD_SIGNAL_BIG_TABLES,DEBEZIUM.CDC_HEARTBEAT_WAL_LT
column.exclude.list: WALLET.SYMCREDITCARD.CC_NUMBER,WALLET.SYMDIRECTDEBIT.CUSTOMER_BANK_ACCT_NUMBER
topic.prefix: ebiz_LT_wallet_big_tables
topic.creation.enable : true
topic.creation.default.replication.factor: -1
topic.creation.default.partitions: -1

schema.history.internal.kafka.topic: wallet_LT.big_table_schema_history
schema.history.internal.consumer.security.protocol: SASL_SSL
schema.history.internal.consumer.sasl.mechanism: SCRAM-SHA-512
schema.history.internal.consumer.sasl.jaas.config: 'org.apache.kafka.common.security.scram.ScramLoginModule required username=cdo password=${secrets:dba-debezium/msk-secret-kafka:kafka-connect-msk-password};'
schema.history.internal.producer.security.protocol: SASL_SSL
schema.history.internal.producer.sasl.mechanism: SCRAM-SHA-512
schema.history.internal.producer.sasl.jaas.config: 'org.apache.kafka.common.security.scram.ScramLoginModule required username=cdo password=${secrets:dba-debezium/msk-secret-kafka:kafka-connect-msk-password};'

# Signal configuration for incremental snapshots
signal.kafka.bootstrap.servers: <boot strap server names>
signal.kafka.topic: ebiz_wallet_big_table_LT_signal
signal.enabled.channels: source,kafka
signal.data.collection: DB3L231.debezium.debezium_wal_load_signal_big_tables
signal.consumer.security.protocol: SASL_SSL
signal.consumer.sasl.mechanism: SCRAM-SHA-512
signal.consumer.sasl.jaas.config: org.apache.kafka.common.security.scram.ScramLoginModule required username=cdo password=${secrets:dba-debezium/msk-secret-kafka:kafka-connect-msk-password};
signal.producer.security.protocol: SASL_SSL
signal.producer.sasl.mechanism: SCRAM-SHA-512
signal.producer.sasl.jaas.config: org.apache.kafka.common.security.scram.ScramLoginModule required username=cdo password=${secrets:dba-debezium/msk-secret-kafka:kafka-connect-msk-password};

notification.enabled.channels: sink
notification.sink.topic.name: ebiz_wallet_LT_notification_big_tables

#heartbeat parameters
heartbeat.action.query: UPDATE debezium.CDC_HEARTBEAT_WAL_LT SET last_update=SYSTIMESTAMP WHERE id=3

When we issue a signal through the database source signal table, nothing happens.

INSERT INTO debezium.debezium_wal_load_signal_big_tables (id, type, data)
VALUES (
  'backfill-2025-09-25-01',
  'execute-snapshot',
  '{"type":"incremental","data-collections":["DB3L231.WALLET.SYMADDRESS"]}'
);

When we publish a message to the signal topic, we see the signal processor picks up the message tries to chunk the table but does nothing after that. It puts water marking entries into the source signal table though. 

We did see couple of messages successfully exporting first 10k records but don't see these messages landed up in the kafka topic (though we see the CDC records coming in for this table).

{"stream":"stdout","timestamp":1758839824568,"log":{"method":"createDataEventsForTable","@timestamp":"2025-09-25T22:37:04.568Z","logger_name":"io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource","source_host":"kafka-connect-dba-ebiz-wallet-connect-0","line_number":"630","message":"\t Finished exporting 10000 records for window of table table 'DB3L231.WALLET.SYMADDRESS'; total duration '00:00:00.408'","class":"io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource","file":"AbstractIncrementalSnapshotChangeEventSource.java","@version":1.0,"level":"DEBUG","thread_name":"debezium-oracleconnector-ebiz_LT_wallet_big_tables-change-event-source-coordinator","mdc":{"dbz.connectorContext":"streaming","connector.context":"[debezium-connector-ebiz-walletload-big-tables|task-0] ","dbz.connectorName":"ebiz_LT_wallet_big_tables","dbz.databaseName":"DB3L231","dbz.connectorType":"Oracle","dbz.taskId":"0"}}}

backfill-2025-09-25-01 execute-snapshot {"type":"incremental","data-collections":["DB3L231.WALLET.SYMADDRESS"]}
backfill-2025-09-25-02 execute-snapshot {"type":"incremental","data-collections":["WALLET.SYMADDRESS"]}
755a94b6-bc44-4729-8064-c9bd0f78adf1-open snapshot-window-open {"openWindowTimestamp": "2025-09-25T20:22:27.355627355Z"}
755a94b6-bc44-4729-8064-c9bd0f78adf1-close snapshot-window-close {"openWindowTimestamp": "2025-09-25T20:22:27.355627355Z", "closeWindowTimestamp": "2025-09-25T20:22:28.189102767Z"}
73bff102-c8d4-497c-9b24-7294aaeda1cf-open snapshot-window-open {"openWindowTimestamp": "2025-09-25T20:27:47.348043699Z"}
73bff102-c8d4-497c-9b24-7294aaeda1cf-close snapshot-window-close {"openWindowTimestamp": "2025-09-25T20:27:47.348043699Z", "closeWindowTimestamp": "2025-09-25T20:27:48.003056887Z"}
eb8b0682-9bbe-4080-b2f9-7e079875739c-open snapshot-window-open {"openWindowTimestamp": "2025-09-25T21:05:03.818012128Z"}
eb8b0682-9bbe-4080-b2f9-7e079875739c-close snapshot-window-close {"openWindowTimestamp": "2025-09-25T21:05:03.818012128Z", "closeWindowTimestamp": "2025-09-25T21:05:04.312305218Z"}
d675076a-8112-484e-ac75-ae9f45097649-open snapshot-window-open {"openWindowTimestamp": "2025-09-25T22:37:04.092672301Z"}
d675076a-8112-484e-ac75-ae9f45097649-close snapshot-window-close {"openWindowTimestamp": "2025-09-25T22:37:04.092672301Z", "closeWindowTimestamp": "2025-09-25T22:37:04.568646424Z"}

Are we missing anything here?

Logs since we started the connector earlier today is attached here.

Thanks again.

Regards,
Shahul Nagoorkani
search-results-2025-09-25T16_50_14.212-0700.csv.zip

Shahul Nagoorkani

unread,
Sep 25, 2025, 10:05:52 PM (6 days ago) Sep 25
to debezium
Issue seems to be fixed after changing:

signal.data.collection: DB3L231.debezium.debezium_wal_load_signal_big_tables

to
signal.data.collection: DB3L231.DEBEZIUM.DEBEZIUM_WAL_LOAD_SIGNAL_BIG_TABLES

So if we specify multiple tables in the signal, will the connector start the incremental load for multiple tables at the same time going by the snapshot.max.threads: 3?

Regards,
Shahul Nagoorkani

Chris Cranford

unread,
Sep 26, 2025, 12:40:09 AM (6 days ago) Sep 26
to debe...@googlegroups.com
Hi -

At this time, incremental snapshots does not use multiple threads, so sending a signal with multiple tables will append those tables to the snapshot collection list in the order they're defined in the signal and they'll be snapshotted one after the other in sequential order.

The `snapshot.max.threads` configuration property is only applicable for the initial and ad-hoc blocking snapshots only.

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 visit https://groups.google.com/d/msgid/debezium/46cd6499-64d2-46e0-ac52-a5edf562a1b5n%40googlegroups.com.

Shahul Nagoorkani

unread,
Sep 26, 2025, 1:38:38 AM (6 days ago) Sep 26
to debezium
Thanks Chris for your insights.

We have several larger tables in TBs(5 to 10 TBs) that needs to be replicated and the incremental snaphsot though reliable is slow with it not providing parallelism. The one that's running right now so far copied around only 27M out of 1.3B records in last 3 hours. 

We are planning to start multiple connectors atleast 5 in parallel. But even with that 5 parallel threads, it would probably take several days to complete. Are there any suggestions around make the loads faster? 

Thanks.
Shahul Nagoorkani

Chris Cranford

unread,
Sep 26, 2025, 7:27:49 AM (6 days ago) Sep 26
to debe...@googlegroups.com
Hi,

In situations where users have very large environments, we have had some chose to use an external option for the initial data load. For example:

    - Start the connector on the source with `snapshot.mode` set to `no_data`.
    - Once data begins to stream, perform your SQL dump from the source using an external tool.
    - Load the dump data into the target system.
    - Deploy the sink connector to bring the target up to date with the source

Are those larger tables, do they have CLOB or BLOB columns or just a lot of VARCHAR2 columns? And do you have EXTENDED max_string_size enabled on the Oracle database?

Thanks,
-cc
Reply all
Reply to author
Forward
0 new messages