-
Bug
-
Resolution: Unresolved
-
Major
-
3.0.9.Final
-
None
-
False
-
-
False
-
Important
In order to make your issue reports as actionable as possible, please provide the following information, depending on the issue type.
Bug report
For bug reports, provide this information, please:
What Debezium connector do you use and what version?
Mongo connector, 3.0.7.Final
What is the connector configuration?
snippet:
"capture.mode": "change_streams_update_full", "capture.scope": "database", "incremental.snapshot.chunk.size": 1024, "incremental.snapshot.watermarking.strategy": "insert_delete", "signal.enabled.channels": "source", "signal.data.collection": "signals_collection", "snapshot.fetch.size": "20480", "snapshot.mode": "when_needed",
What is the captured database version and mode of deployment?
MongoDB Atlas, replica set, version 7.0, managed
What behavior do you expect?
Incremental snapshot runs to completion.
What behavior do you see?
During incremental snapshot execution, connector hits StackOverflowError. Connector stops running. Manually resuming resolves the issue temporarily, but eventually it halts on a stack overflow again.
Do you see the same behaviour using the latest released Debezium version?
N/A, this repros in production. We have not updated to latest yet. But this was confirmed as an issue resolved for relational databases but not yet resolved for Mongo.
Do you have the connector logs, ideally from start till finish?
[2025-10-07 16:03:40,770] ERROR WorkerSourceTask{id=mongo_mfa_dwh-0} Task threw an uncaught and unrecoverable exception. Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask)
org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)
at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:150)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
at java.base/java.util.concurrent.FutureTask.run(Unknown Source)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
at java.base/java.lang.Thread.run(Unknown Source)
Caused by: java.lang.StackOverflowError
at java.base/java.lang.ClassLoader.defineClass1(Native Method)
at java.base/java.lang.ClassLoader.defineClass(Unknown Source)
at java.base/java.security.SecureClassLoader.defineClass(Unknown Source)
at java.base/java.net.URLClassLoader.defineClass(Unknown Source)
at java.base/java.net.URLClassLoader$1.run(Unknown Source)
at java.base/java.net.URLClassLoader$1.run(Unknown Source)
at java.base/java.security.AccessController.doPrivileged(Unknown Source)
at java.base/java.net.URLClassLoader.findClass(Unknown Source)
at org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:116)
at java.base/java.lang.ClassLoader.loadClass(Unknown Source)
at io.debezium.connector.mongodb.connection.MongoDbConnections.lambda$eventSourcingErrorHandler$1(MongoDbConnections.java:50)
at io.debezium.connector.mongodb.connection.MongoDbConnection.execute(MongoDbConnection.java:111)
at io.debezium.connector.mongodb.connection.MongoDbConnection.execute(MongoDbConnection.java:88)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowOpen(MongoDbIncrementalSnapshotChangeEventSource.java:216)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:291)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
at io.debezium.connector.mongodb.snapshot.MongoDbDeleteWindowCloser.closeWindow(MongoDbDeleteWindowCloser.java:55)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowClose(MongoDbIncrementalSnapshotChangeEventSource.java:238)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:341)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
at io.debezium.connector.mongodb.snapshot.MongoDbDeleteWindowCloser.closeWindow(MongoDbDeleteWindowCloser.java:55)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowClose(MongoDbIncrementalSnapshotChangeEventSource.java:238)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:341)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
at io.debezium.connector.mongodb.snapshot.MongoDbDeleteWindowCloser.closeWindow(MongoDbDeleteWindowCloser.java:55)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowClose(MongoDbIncrementalSnapshotChangeEventSource.java:238)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:341)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
at io.debezium.connector.mongodb.snapshot.MongoDbDeleteWindowCloser.closeWindow(MongoDbDeleteWindowCloser.java:55)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowClose(MongoDbIncrementalSnapshotChangeEventSource.java:238)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:341)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
at io.debezium.connector.mongodb.snapshot.MongoDbDeleteWindowCloser.closeWindow(MongoDbDeleteWindowCloser.java:55)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowClose(MongoDbIncrementalSnapshotChangeEventSource.java:238)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:341)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
at io.debezium.connector.mongodb.snapshot.MongoDbDeleteWindowCloser.closeWindow(MongoDbDeleteWindowCloser.java:55)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowClose(MongoDbIncrementalSnapshotChangeEventSource.java:238)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:341)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
at io.debezium.connector.mongodb.snapshot.MongoDbDeleteWindowCloser.closeWindow(MongoDbDeleteWindowCloser.java:55)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowClose(MongoDbIncrementalSnapshotChangeEventSource.java:238)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:341)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
at io.debezium.connector.mongodb.snapshot.MongoDbDeleteWindowCloser.closeWindow(MongoDbDeleteWindowCloser.java:55)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.emitWindowClose(MongoDbIncrementalSnapshotChangeEventSource.java:238)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.readChunk(MongoDbIncrementalSnapshotChangeEventSource.java:341)
at io.debezium.connector.mongodb.snapshot.MongoDbIncrementalSnapshotChangeEventSource.closeWindow(MongoDbIncrementalSnapshotChangeEventSource.java:133)
...
How to reproduce the issue using our tutorial deployment?
<Your answer>
- relates to
-
DBZ-8011 StackOverflow exception on incremental snapshot
-
- Closed
-