HuG0012 opened a new issue, #10470: URL: https://github.com/apache/seatunnel/issues/10470
### Search before asking - [x] I had searched in the [issues](https://github.com/apache/seatunnel/issues?q=is%3Aissue+label%3A%22bug%22) and found no similar issues. ### What happened In mongo cdc, I set config heartbeat.interval.ms to a non-zero value to enable heartbeats. And I got NPE. ``` @Override public boolean isRecordBetween( SourceRecord record, @Nonnull Object[] splitStart, @Nonnull Object[] splitEnd) { BsonDocument documentKey = getDocumentKey(record); // getDocumentKey returns null BsonDocument splitKeys = (BsonDocument) splitStart[0]; String firstKey = splitKeys.getFirstKey(); BsonValue keyValue = documentKey.get(firstKey); // Here the documentKey is null BsonValue lowerBound = ((BsonDocument) splitStart[1]).get(firstKey); BsonValue upperBound = ((BsonDocument) splitEnd[1]).get(firstKey); if (isFullRange(lowerBound, upperBound)) { return true; } return isValueInRange(lowerBound, keyValue, upperBound); } ``` <img width="529" height="261" alt="Image" src="https://github.com/user-attachments/assets/5c9ced03-919c-4f3e-8d3e-ce3ee09d777d" /> ``` public static BsonDocument extractBsonDocument( Struct value, @Nonnull Schema valueSchema, String fieldName) { if (valueSchema.field(fieldName) != null) { // fieldName is documentKey, and the schema of record has no documentKey. String docString = value.getString(fieldName); if (docString != null) { return BsonDocument.parse(docString); } } return null; } ``` ### SeaTunnel Version 2.3.9 ### SeaTunnel Config ```conf env { # You can set engine configuration here parallelism = 1 job.mode = "STREAMING" checkpoint.interval = 5000 } source { MongoDB-CDC { hosts = "your_host:port" database = ["log_super"] collection = ["log_super.USER_TRACE_LOG"] username = password = heartbeat.interval.ms = 30000 schema = { table = "log_super.USER_TRACE_LOG" fields { "_id" : string, "stepMessage" : string, "stepResult" : string } } } } # Console printing of the read Mongodb data sink { Console {} } ``` ### Running Command ```shell none ``` ### Error Exception ```log [1073123856090136577] 2026-02-09 14:19:22,053 ERROR org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcherManager - Received uncaught exception. java.lang.RuntimeException: SplitFetcher thread 0 received unexpected exception while polling the records at org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:165) ~[classes/:?] at org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcher.run(SplitFetcher.java:81) [classes/:?] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_462] at java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:266) [?:1.8.0_462] at java.util.concurrent.FutureTask.run(FutureTask.java) [?:1.8.0_462] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_462] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_462] at java.lang.Thread.run(Thread.java:750) [?:1.8.0_462] Caused by: java.lang.NullPointerException at org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch.MongodbFetchTaskContext.isRecordBetween(MongodbFetchTaskContext.java:161) ~[classes/:?] at org.apache.seatunnel.connectors.cdc.base.source.reader.external.IncrementalSourceScanFetcher.isChangeRecordInChunkRange(IncrementalSourceScanFetcher.java:258) ~[classes/:?] at org.apache.seatunnel.connectors.cdc.base.source.reader.external.IncrementalSourceScanFetcher.pollSplitRecordsIfExactlyOnce(IncrementalSourceScanFetcher.java:185) ~[classes/:?] at org.apache.seatunnel.connectors.cdc.base.source.reader.external.IncrementalSourceScanFetcher.pollSplitRecords(IncrementalSourceScanFetcher.java:121) ~[classes/:?] at org.apache.seatunnel.connectors.cdc.base.source.reader.IncrementalSourceSplitReader.fetch(IncrementalSourceSplitReader.java:75) ~[classes/:?] at org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.FetchTask.run(FetchTask.java:54) ~[classes/:?] at org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:162) ~[classes/:?] ... 7 more ``` ### Zeta or Flink or Spark Version _No response_ ### Java or Scala Version _No response_ ### Screenshots _No response_ ### Are you willing to submit PR? - [ ] Yes I am willing to submit a PR! ### Code of Conduct - [x] I agree to follow this project's [Code of Conduct](https://www.apache.org/foundation/policies/conduct) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
