-
Bug
-
Resolution: Done
-
Major
-
None
-
None
-
False
-
None
-
False
Bug report
In a Debezium connector which was getting data from tables smoothly, it suddenly stopped after the error message: Error Msg = ORA-01001: invalid cursor\nORA-01001: invalid cursor\n\n\tat
After this there was no self recovery attempt for an hour, and we had to end up restarting the connector to make messages flow again.
{"@timestamp":"2024-10-02T20:47:46.094Z","ecs.version":"1.2.0","log.level":"INFO","message":"Stopped KafkaOffsetBackingStore","process.thread.name":"pool-6-thread-1","log.logger":"org.apache.kafka.connect.storage.KafkaOffsetBackingStore"}{"@timestamp":"2024-10-02T20:47:46.095Z","ecs.version":"1.2.0","log.level":"ERROR","message":"Error while trying to run connector class 'io.debezium.connector.oracle.OracleConnector'","process.thread.name":"pool-6-thread-1","log.logger":"io.debezium.embedded.EmbeddedEngine","error.type":"org.apache.kafka.connect.errors.ConnectException","error.message":"An exception occurred in the change event producer. This connector will be stopped.","error.stack_trace":"org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.\n\tat io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)\n\tat io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:264)\n\tat io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)\n\tat io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:313)\n\tat io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:203)\n\tat io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:143)\n\tat java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)\n\tat java.base/java.util.concurrent.FutureTask.run(Unknown Source)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)\n\tat java.base/java.lang.Thread.run(Unknown Source)\nCaused by: java.sql.SQLException: ORA-01001: invalid cursor\nORA-01001: invalid cursor\n\n\tat oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:630)\n\tat oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:564)\n\tat oracle.jdbc.driver.T4C8Oall.processError(T4C8Oall.java:1151)\n\tat oracle.jdbc.driver.T4CTTIfun.receive(T4CTTIfun.java:771)\n\tat oracle.jdbc.driver.T4CTTIfun.doRPC(T4CTTIfun.java:299)\n\tat oracle.jdbc.driver.T4C8Oall.doOALL(T4C8Oall.java:498)\n\tat oracle.jdbc.driver.T4CPreparedStatement.doOall8(T4CPreparedStatement.java:152)\n\tat oracle.jdbc.driver.T4CPreparedStatement.executeForDescribe(T4CPreparedStatement.java:937)\n\tat oracle.jdbc.driver.OracleStatement.prepareDefineBufferAndExecute(OracleStatement.java:1172)\n\tat oracle.jdbc.driver.OracleStatement.executeMaybeDescribe(OracleStatement.java:1101)\n\tat oracle.jdbc.driver.OracleStatement.executeSQLSelect(OracleStatement.java:1426)\n\tat oracle.jdbc.driver.OracleStatement.doExecuteWithTimeout(OracleStatement.java:1309)\n\tat oracle.jdbc.driver.OraclePreparedStatement.executeInternal(OraclePreparedStatement.java:3746)\n\tat oracle.jdbc.driver.OraclePreparedStatement.executeQuery(OraclePreparedStatement.java:3855)\n\tat oracle.jdbc.driver.OraclePreparedStatementWrapper.executeQuery(OraclePreparedStatementWrapper.java:1098)\n\tat io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor.process(AbstractLogMinerEventProcessor.java:237)\n\tat io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:243)\n\t... 9 more\nCaused by: Error : 1001, Position : 0, Sql = SELECT SCN, SQL_REDO, OPERATION_CODE, TIMESTAMP, XID, CSF, TABLE_NAME, SEG_OWNER, OPERATION, USERNAME, ROW_ID, ROLLBACK, RS_ID, STATUS, INFO, SSN, THREAD#, DATA_OBJ#, DATA_OBJV#, DATA_OBJD# FROM V$LOGMNR_CONTENTS WHERE SCN > :1 AND SCN <= :2 AND (((OPERATION_CODE IN (1,2,3,7,34,36,255)) AND (SEG_OWNER IS NULL OR SEG_OWNER NOT IN ('APPQOSSYS','AUDSYS','CTXSYS','DVSYS','DBSFWUSER','DBSNMP','GGSHAREDCAP','GSMADMIN_INTERNAL','LBACSYS','MDSYS','OJVMSYS','OLAPSYS','ORDDATA','ORDSYS','OUTLN','SYS','SYSTEM','VECSYS','WMSYS','XDB'))) OR (OPERATION_CODE = 5 AND INFO NOT LIKE 'INTERNAL DDL%')), OriginalSql = SELECT SCN, SQL_REDO, OPERATION_CODE, TIMESTAMP, XID, CSF, TABLE_NAME, SEG_OWNER, OPERATION, USERNAME, ROW_ID, ROLLBACK, RS_ID, STATUS, INFO, SSN, THREAD#, DATA_OBJ#, DATA_OBJV#, DATA_OBJD# FROM V$LOGMNR_CONTENTS WHERE SCN > ? AND SCN <= ? AND (((OPERATION_CODE IN (1,2,3,7,34,36,255)) AND (SEG_OWNER IS NULL OR SEG_OWNER NOT IN ('APPQOSSYS','AUDSYS','CTXSYS','DVSYS','DBSFWUSER','DBSNMP','GGSHAREDCAP','GSMADMIN_INTERNAL','LBACSYS','MDSYS','OJVMSYS','OLAPSYS','ORDDATA','ORDSYS','OUTLN','SYS','SYSTEM','VECSYS','WMSYS','XDB'))) OR (OPERATION_CODE = 5 AND INFO NOT LIKE 'INTERNAL DDL%')), Error Msg = ORA-01001: invalid cursor\nORA-01001: invalid cursor\n\n\tat oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:637)\n\t... 25 more\n"}
{"@timestamp":"2024-10-02T20:48:43.414Z","ecs.version":"1.2.0","log.level":"INFO","message":"[AdminClient clientId=debezium-servershared-admin] Node 1056 disconnected.","process.thread.name":"kafka-admin-client-thread | debezium-servershared-admin","log.logger":"org.apache.kafka.clients.NetworkClient"}
What Debezium connector do you use and what version?
It is 2.7.3, but here are helpful additional library information we use in our Java Springboot application.
implementation 'io.debezium:debezium-api:2.7.3.Final'
implementation'io.debezium:debezium-embedded:2.7.3.Final'
implementation 'org.apache.kafka:connect-runtime:3.6.1'
implementation 'io.debezium:debezium-connector-oracle:2.7.3.Final'
implementation 'com.oracle.database.jdbc:ojdbc8:21.6.0.0'
implementation 'io.debezium:debezium-storage-kafka:2.7.3.Final'
What is the connector configuration?
connector:
decimalHandlingMode: string
name: <<NAME>>
class: io.debezium.connector.oracle.OracleConnector
offsetTopic: <<OFFSETTOPICNAME>>
offsetTopicNumberOfPartitions: 1
offsetTopicReplicationFactor: 1
heartbeatIntervalInMsecs: 100000
heartbeatQuery: SELECT * FROM MYTABLE WHERE ROWNUM <= 1
.with("name", wmsDbzConnectorName)
.with("connector.class", wmsDbzConnectorClass)
.with("offset.storage", "org.apache.kafka.connect.storage.KafkaOffsetBackingStore")
.with("offset.storage.topic", debeziumConnectorOffsetTopic)
.with(DistributedConfig.OFFSET_STORAGE_PARTITIONS_CONFIG, debeziumConnectorOffsetTopicPartitions)
.with(DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, debeziumConnectorOffsetTopicReplicationFactor)
.with("offset.flush.interval.ms", "60000")
.with("database.hostname", wmsDbHost)
.with("database.port", port)
.with("database.user", username)
.with("database.password", password)
.with("database.dbname", dbName)
.with("schema.include.list", schemaList)
.with("table.include.list", tableList)
.with("include.schema.changes", "false")
.with("topic.prefix", topicPrefix)
.with("database.server.name", dbserverName)
.with("snapshot.mode", snapshotMode) // It is set as initial
.with("converter.schemas.enable", "false")
.with("decimal.handling.mode", decimalHandlingMode)
.with("heartbeat.interval.ms", heartbeatInterval)
.with("heartbeat.action.query", heartbeatActionQuery)
.with("database.history.kafka.topic", dbAndSchemaHistoryTopic)
.with("database.history.consumer.group.id", dbAndSchemaHistoryTopicConsumerGroupId)
.with("schema.history.internal.kafka.topic", schemaTopic)
.with("schema.history.internal.kafka.bootstrap.servers", schemaBootstrapServers)
.with("schema.history.internal.consumer.security.protocol", schemaSecurityProtocol)
.with("schema.history.internal.consumer.ssl.keystore.type", schemaSslKeyStoreType)
.with("schema.history.internal.consumer.ssl.keystore.location", schemaSslKeystoreLocation)
.with("schema.history.internal.consumer.ssl.keystore.password", schemaSslKeystorePassword)
.with("schema.history.internal.consumer.ssl.truststore.type", schemaSslTrustStoreType)
.with("schema.history.internal.consumer.ssl.truststore.location", schemaSslTruststoreLocation)
.with("schema.history.internal.consumer.ssl.truststore.password", schemaSslTruststorePassword)
.with("schema.history.internal.consumer.ssl.endpoint.identification.algorithm", sslEndpointAlgorithm)
.with("schema.history.internal.producer.security.protocol", schemaSecurityProtocol)
.with("schema.history.internal.producer.ssl.keystore.type", schemaSslKeyStoreType)
.with("schema.history.internal.producer.ssl.keystore.location", schemaSslKeystoreLocation)
.with("schema.history.internal.producer.ssl.keystore.password", schemaSslKeystorePassword)
.with("schema.history.internal.producer.ssl.truststore.type", schemaSslTrustStoreType)
.with("schema.history.internal.producer.ssl.truststore.location", schemaSslTruststoreLocation)
.with("schema.history.internal.producer.ssl.truststore.password", schemaSslTruststorePassword)
.with("schema.history.internal.producer.ssl.endpoint.identification.algorithm", sslEndpointAlgorithm)
.with("bootstrap.servers", schemaBootstrapServers)
.with("security.protocol", schemaSecurityProtocol)
.with("ssl.keystore.location", schemaSslKeystoreLocation)
.with("ssl.keystore.password", schemaSslKeystorePassword)
.with("ssl.truststore.location", schemaSslTruststoreLocation)
.with("ssl.truststore.password", schemaSslTruststorePassword)
.with("ssl.endpoint.identification.algorithm", sslEndpointAlgorithm)
What is the captured database version and mode of deployment?
(E.g. on-premises, with a specific cloud provider, etc.)
Data base in running on a data center and is an Oracle database of version 19c.
DB Server on Dev Database runs on a 2 node RAC. The Archived log files (which Log Miner reads from) are shared to both nodes in RAC as a shared path.
What behavior do you expect?
Invalid cursor error should have either not arrived at all OR even if it arrives, it shouldn't have stopped for ever and should have restarted on its own instead of doing a manual restart to get the messages flow working fine again.
What behavior do you see?
Invalid cursor error shown above stopped the Connector all together and there was no attempt to recover.
Do you see the same behaviour using the latest released Debezium version?
Unsure if it happened before as this is the first time we managed to capture this error as the cause of failure by not letting logs get overwritten.
Do you have the connector logs, ideally from start till finish?
The trace given in Bug report might be enough.
How to reproduce the issue using our tutorial deployment?
Unsure. Just happened like that in the middle.
Feature request or enhancement
Maybe if the connector stops due to this reason, a retry will be good or a self restart attempt.
Which use case/requirement will be addressed by the proposed feature?
This is a blocking issue
Implementation ideas (optional)
Try to get it restarted if such an issue happens.