-
Bug
-
Resolution: Done
-
Major
-
3.0.8.Final, 3.1.0.Final
-
None
-
1
-
False
-
-
False
-
-
Given this configuration:
{
"connector.class": "io.debezium.connector.oracle.OracleConnector",
"database.user": "c##ggadmin",
"database.dbname": "dm02cdb",
"database.pdb.name": "PDB_DSCSM",
"transforms": "unwrap",
"schema.history.internal.kafka.bootstrap.servers": "p-xx-xx-001:9092,p-xx-xx-002:9092,p-xx-xx-003:9092",
"log.mining.strategy": "online_catalog",
"database.port": "1521",
"schema.history.internal.store.only.captured.tables.ddl": "true",
"value.converter.schema.registry.url": "http://p-xx-xx-001:8081,http://p-xx-xx-002:8081,http://p-xx-xx-003:8081",
"topic.prefix": "PROD",
"decimal.handling.mode": "double",
"schema.history.internal.kafka.topic": "dbhistory_pdb_dscsm",
"log.mining.continuous.mine": "true",
"database.hostname": "xx.xx.xx.xx",
"database.password": "******",
"name": "source-pdb_dscsm-connector",
"transforms.unwrap.type": "io.debezium.transforms.ExtractNewRecordState",
"log.mining.archive.log.only.mode": "true",
"table.include.list": "DSCSM.CSM_SRV_ORDER_COURSE,DSCSM.CSM_MD_PROCESS_TOKEN,DSCSM.INV_INV_ORDER_LINES,DSCSM.CSM_SRV_DISPATCH_HISTORY,DSCSM.INV_EX_ORDER_LOGISTIC,DSCSM.INV_EX_ORDER_HEADERS,DSCSM.INV_DELIVER_ORDER_HEADERS,DSCSM.INV_INV_ORDER_HEADERS,DSCSM.CSM_SRV_ACCESS_DETAIL,DSCSM.DC_SERIAL_NO,DSCSM.CSM_SRV_ACCESS_HEADER,DSCSM.CSM_SRV_VISIT_GRADE,DSCSM.CSM_BRM_PROCESS_RECORD,DSCSM.CSM_SRV_ORDER_DETAIL,DSCSM.CSM_SRV_ORDER_DETAIL,DSCSM.CSM_SRV_ORDER,DSCSM.CSM_SPM_MATCHING_LINE,DSCSM.DC_DATA_ITEM_CSM,DSCSM.CSM_MD_FINAL_CUS,DSCSM.CSM_MD_FINAL_CUS_CONTACT,DSCSM.CSM_SPM_MATCHING_HEADER,DSCSM.CSM_MD_FINAL_CUS_ADDRESS,DSCSM.CSM_SRV_ORDER_EXTEND,DSCSM.TMS_RETURN_INFO,DSCSM.CSM_BRM_FAC_DO_LINES,DSCSM.CSM_BRM_FAC_DO_INFO,DSCSM.INV_SN_LOCK,DSCSM.CSM_SRV_ACCESS_RECEIVE,DSCSM.CSM_SYS_USER,DSCSM.CSM_SRV_ORDER_BACK_APPLY,DSCSM.CSM_BRM_APPLY_CUSTOMER,DSCSM.CSM_BRM_APPLY_LINES,DSCSM.CSM_BRM_APPLY_HEADER,DSCSM.CSM_BRM_HEADER,DSCSM.INV_INVENTORY,DSCSM.CSM_BRM_CUSTOM_STEP,DSCSM.CSM_BRM_DOA_APPLY_HEADER,DSCSM.CSM_MD_AREA_MAPPING,DSCSM.CSM_MD_FAULT,DSCSM.CSM_MD_SERVICE_PROVIDER,DSCSM.CSM_MD_SRV_CONTENT,DSCSM.CSM_MD_SRV_HIERARCHY,DSCSM.CSM_SPM_LEAVE_FACTORY_HEADER,DSCSM.CSM_SPM_LEAVE_FACTORY_LINE,DSCSM.INV_DELIVER_ORDER_BARS,DSCSM.INV_DELIVER_ORDER_LINES,DSCSM.INV_EX_ORDER_LINES,DSCSM.INV_BUSINESS_ORDER_TYPE",
"value.converter": "io.confluent.connect.avro.AvroConverter",
"key.converter": "io.confluent.connect.avro.AvroConverter",
"key.converter.schema.registry.url": "http://p-xx-xx-001:8081,http://p-xx-xx-002:8081,http://p-xx-xx-003:8081",
"snapshot.mode": "no_data"
}
the following happens on Oracle 19 and Debezium 3.1:
[2025-04-10 14:32:09,333] INFO [source-pdb_dscsm-connector|task-0] Group #46: 8589934592 bytes (io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource:300)
[2025-04-10 14:32:10,285] ERROR [source-pdb_dscsm-connector|task-0] Mining session stopped due to error. (io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource:262)
java.lang.NullPointerException: Cannot invoke "java.util.List.stream()" because "this.currentLogFiles" is null
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.updateRedoLogMetrics(LogMinerStreamingChangeEventSource.java:515)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.initializeRedoLogsForMining(LogMinerStreamingChangeEventSource.java:416)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:179)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)
at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:324)
at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:203)
at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:143)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
[2025-04-10 14:32:10,286] ERROR [source-pdb_dscsm-connector|task-0] Producer failure (io.debezium.pipeline.ErrorHandler:52)
java.lang.NullPointerException: Cannot invoke "java.util.List.stream()" because "this.currentLogFiles" is null
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.updateRedoLogMetrics(LogMinerStreamingChangeEventSource.java:515)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.initializeRedoLogsForMining(LogMinerStreamingChangeEventSource.java:416)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:179)
at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)
at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:324)
at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:203)
at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:143)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
The issue is that users shouldn't use continuous mining with Oracle 18 or later, but we should warn when doing so and correctly ignore the setting if Oracle is 18 or greater so that log lists are correctly generated.
- links to
-
RHEA-2025:154266
Red Hat build of Debezium 3.2.4 release