Debezium 2.5 for Mongodb 4.4 - Incremental snapshot not working on signal

20 views
Skip to first unread message

Quan Le Anh

unread,
Nov 3, 2025, 11:58:25 PM (10 days ago) Nov 3
to debezium

Hi community,

I’m preparing to migrate data from a source MongoDB (4.4) to a target database in production. I’m currently testing this setup in a lab environment. The migration strategy involves:

  • Using Debezium connector for CDC to keep both databases in sync
  • Using incremental snapshots on-demand whenever there’s a miss-sync, without having to snapshot the entire database again
Problem

I’m trying to use Debezium (v2.5) MongoDB connector to perform on-demand incremental snapshots via Kafka signals. The connector receives the signal and creates a document in the cdc.signal collection, but no snapshot is triggered - no Kafka topics are created for the snapshot data and no data is exported.

CDC works fine for new records (I tested by inserting new records), but the incremental snapshot signal doesn’t work.

Setup

Connector Configuration:

{ "name": "debezium-mongodb-source-connector-3", "config": { "connector.class": "io.debezium.connector.mongodb.MongoDbConnector", "tasks.max": "1", "mongodb.connection.string": "mongodb://mongodb-source:27017/?replicaSet=rs0", "topic.prefix": "mongo-source-v2", "database.include.list": "sample_mflix", "snapshot.mode": "never", "incremental.snapshot.watermarking.strategy": "insert_insert", "incremental.snapshot.chunk.size": "8192", "incremental.snapshot.allow.schema.changes": "true", "key.converter": "org.apache.kafka.connect.json.JsonConverter", "key.converter.schemas.enable": "true", "value.converter": "org.apache.kafka.connect.json.JsonConverter", "value.converter.schemas.enable": "true", "producer.override.max.request.size": "52428800", "signal.enabled.channels": "kafka", "signal.kafka.topic": "debezium_signals.debezium-mongodb-source-connector-2", "signal.kafka.bootstrap.servers": "kafka:29092", "signal.data.collection": "cdc.signal" } }

Signal Message Sent to Kafka:

{ "type": "execute-snapshot", "data": { "data-collections": [ "sample_mflix.comments", "sample_mflix.movies", "sample_mflix.sessions", "sample_mflix.theaters", "sample_mflix.users" ], "type": "incremental" } }

Document Created in MongoDB cdc.signal Collection:

{ "_id": "d30dae13-a99b-493e-87bc-d9a73826535b-open", "type": "snapshot-window-open", "payload": "" } What Happens
  1. Signal is sent to Kafka topic debezium_signals.debezium-mongodb-source-connector-2
  2. Connector creates a document in MongoDB cdc.signal collection with type snapshot-window-open
  3. Logs show the signal is received and snapshot processing starts, but no Kafka topics are created for the snapshot data
Logs [2025-11-03 10:04:47,444] INFO Requested 'INCREMENTAL' snapshot of data collections '[sample_mflix.comments, sample_mflix.movies, sample_mflix.sessions, sample_mflix.theaters, sample_mflix.users]' with additional conditions '[]' and surrogate key 'PK of table will be used' (io.debezium.pipeline.signal.actions.snapshotting.ExecuteSnapshot) [2025-11-03 10:04:47,445] DEBUG Starting new chunk with id 'ccbbaa66-a511-4c97-941d-2df836aa1ff9' (io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotContext) [2025-11-03 10:04:47,494] DEBUG For collection 'rs0.sample_mflix.comments' using query: '{"_id": {"$lte": {"$oid": "5b7327c34a68c3f7a7bcb1d3"}}}', key: 'null', maximum key: '[5b7327c34a68c3f7a7bcb1d3]' to get all _id fields (io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource) [2025-11-03 10:04:47,517] DEBUG For collection chunk, 'rs0.sample_mflix.comments' using query: '{"_id": {"$lte": {"$oid": "5a9427648b0beebeb695dacc"}}}', key: 'null', maximum key: '[5a9427648b0beebeb695dacc]' (io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource) [2025-11-03 10:04:47,617] DEBUG Next window will resume from [5a9427648b0beebeb695dacc] (io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource) [2025-11-03 10:04:47,618] DEBUG Finished exporting 8192 records for window of collection 'rs0.sample_mflix.comments'; total duration '00:00:00.126' (io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource) [2025-11-03 10:04:47,770] WARN [Producer clientId=connector-producer-debezium-mongodb-source-connector-3-0] Error while fetching metadata with correlation id 3 : {mongo-source-v2.cdc.signal=LEADER_NOT_AVAILABLE} (org.apache.kafka.clients.NetworkClient)

Any help would be greatly appreciated! I’m also open to any best practice recommendations for migrating data in production using this approach.


Chris Cranford

unread,
Nov 4, 2025, 3:57:19 AM (10 days ago) Nov 4
to debe...@googlegroups.com
Hi -

This is a common mistake when configuring a connector to use any of the other signal channels like a Kafka topic. When configuring the `signal.enabled.channels`, it's important that you retain `source` as one of the options when you want to perform incremental snapshots, as the `signal.data.collection` emits open/close signals that are used by the Incremental Snapshot process. 

Please try changing this in your configuration:

    "signal.enabled.channels": "source,kafka"

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/405c2428-e734-4cae-a107-2de92f12d8ben%40googlegroups.com.

Reply all
Reply to author
Forward
0 new messages