Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-7479

PreparedStatement leak in Oracle ReselectColumnsProcessor

XMLWordPrintable

    • False
    • None
    • False

      Bug report

      For bug reports, provide this information, please:

      What Debezium connector do you use and what version?

      debezium-connector-oracle version 2.5.1.Final

      What is the connector configuration?

      {
        "name": "source-test-connector",
        "config": {
          "connector.class": "io.debezium.connector.oracle.OracleConnector",
          "tasks.max": "1",
          "database.hostname": "oracle",
          "database.port": "1521",
          "database.user": "c##dbzuser",
          "database.password": "dbz",
          "database.dbname": "orclcdb",
          "database.pdb.name": "orclpdb1",
          "database.connection.adapter": "logminer",
          "topic.prefix": "dbz",
          "schema.name.adjustment.mode": "avro",
          "table.include.list": "C##DBZUSER.TEST_TABLE",
          "include.schema.changes": "false",
          "schema.history.internal.kafka.bootstrap.servers" : "kafka:9092",
          "schema.history.internal.kafka.topic": "schema-changes.test",
          "heartbeat.interval.ms": "60000",
          "log.mining.strategy": "online_catalog",
          "log.mining.query.filter.mode": "in",
          "post.processors": "reselector",
          "reselector.type": "io.debezium.processors.reselect.ReselectColumnsPostProcessor",
          "reselector.reselect.columns.include.list": "C##DBZUSER.TEST_TABLE:DATA",
          "custom.metric.tags": "connector=source-test-connector",
          "transforms": "unwrap",
          "transforms.unwrap.type": "io.debezium.transforms.ExtractNewRecordState",
          "key.converter": "org.apache.kafka.connect.json.JsonConverter",
          "key.converter.schemas.enable": "false",
          "value.converter": "org.apache.kafka.connect.json.JsonConverter",
          "value.converter.schemas.enable": "false"
        }
      }
      

      What is the captured database version and mode of deployment?

      Oracle Database 19, Docker

      What behaviour do you expect?

      PreparedStatement-s do not leak when Oracle connector is configured with ReselectColumnsPostProcessor.

      What behaviour do you see?

      Each time, Oracle connector creates a new instance of PreparedStatement because value of commit SCN is added directly to SQL query to reselect column values.

      Do you see the same behaviour using the latest relesead Debezium version?

      Yes

      Do you have the connector logs, ideally from start till finish?

      [2024-02-12 13:33:15,732] WARN [source-test-connector|task-0] Failed to re-select row for table ORCLPDB1.C##DBZUSER.TEST_TABLE and key Struct{ID=658752190684} (io.debezium.processors.reselect.ReselectColumnsPostProcessor:156)
      java.sql.SQLException: ORA-01000: maximum open cursors exceeded
      
              at oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:629)
              at oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:563)
              at oracle.jdbc.driver.T4C8Oall.processError(T4C8Oall.java:1150)
              at oracle.jdbc.driver.T4CTTIfun.receive(T4CTTIfun.java:770)
              at oracle.jdbc.driver.T4CTTIfun.doRPC(T4CTTIfun.java:298)
              at oracle.jdbc.driver.T4C8Oall.doOALL(T4C8Oall.java:497)
              at oracle.jdbc.driver.T4CPreparedStatement.doOall8(T4CPreparedStatement.java:151)
              at oracle.jdbc.driver.T4CPreparedStatement.executeForDescribe(T4CPreparedStatement.java:936)
              at oracle.jdbc.driver.OracleStatement.prepareDefineBufferAndExecute(OracleStatement.java:1171)
              at oracle.jdbc.driver.OracleStatement.executeMaybeDescribe(OracleStatement.java:1100)
              at oracle.jdbc.driver.OracleStatement.executeSQLSelect(OracleStatement.java:1425)
              at oracle.jdbc.driver.OracleStatement.doExecuteWithTimeout(OracleStatement.java:1308)
              at oracle.jdbc.driver.OraclePreparedStatement.executeInternal(OraclePreparedStatement.java:3745)
              at oracle.jdbc.driver.OraclePreparedStatement.executeQuery(OraclePreparedStatement.java:3854)
              at oracle.jdbc.driver.OraclePreparedStatementWrapper.executeQuery(OraclePreparedStatementWrapper.java:1097)
              at io.debezium.jdbc.JdbcConnection.prepareQuery(JdbcConnection.java:780)
              at io.debezium.jdbc.JdbcConnection.reselectColumns(JdbcConnection.java:1610)
              at io.debezium.connector.oracle.OracleConnection.lambda$reselectColumns$23(OracleConnection.java:589)
              at io.debezium.connector.oracle.OracleConnection.optionallyDoInContainer(OracleConnection.java:600)
              at io.debezium.connector.oracle.OracleConnection.reselectColumns(OracleConnection.java:589)
              at io.debezium.processors.reselect.ReselectColumnsPostProcessor.apply(ReselectColumnsPostProcessor.java:149)
              at io.debezium.pipeline.EventDispatcher.doPostProcessing(EventDispatcher.java:707)
              at io.debezium.pipeline.EventDispatcher$StreamingChangeRecordReceiver.changeRecord(EventDispatcher.java:483)
              at io.debezium.pipeline.EventDispatcher$2.changeRecord(EventDispatcher.java:300)
              at io.debezium.relational.RelationalChangeRecordEmitter.emitUpdateRecord(RelationalChangeRecordEmitter.java:124)
              at io.debezium.relational.RelationalChangeRecordEmitter.emitChangeRecords(RelationalChangeRecordEmitter.java:53)
              at io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:271)
              at io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor$1.accept(AbstractLogMinerEventProcessor.java:552)
              at io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor$1.accept(AbstractLogMinerEventProcessor.java:504)
              at io.debezium.connector.oracle.logminer.processor.TransactionCommitConsumer.dispatchChangeEvent(TransactionCommitConsumer.java:380)
              at io.debezium.connector.oracle.logminer.processor.TransactionCommitConsumer.accept(TransactionCommitConsumer.java:117)
              at io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor.handleCommit(AbstractLogMinerEventProcessor.java:576)
              at io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor.processRow(AbstractLogMinerEventProcessor.java:382)
              at io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor.processResults(AbstractLogMinerEventProcessor.java:325)
              at io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor.process(AbstractLogMinerEventProcessor.java:253)
              at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:248)
              at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)
              at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:272)
              at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:197)
              at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:137)
              at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
              at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
              at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
              at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
              at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: Error : 1000, Position : 0, Sql = SELECT "DATA" FROM (SELECT * FROM "C##DBZUSER"."TEST_TABLE" AS OF SCN 2751007772674) WHERE ID=:1 , OriginalSql = SELECT "DATA" FROM (SELECT * FROM "C##DBZUSER"."TEST_TABLE" AS OF SCN 2751007772674) WHERE ID=?, Error Msg = ORA-01000: maximum open cursors exceeded
      
              at oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:636)
              ... 44 more
      

      How to reproduce the issue using our tutorial deployment?

      1. Create a new table:

      CREATE TABLE c##dbzuser.test_table (
        id   NUMBER(10) NOT NULL PRIMARY KEY,
        data CLOB
      );
      

      2. Create a new connector with ReselectColumnsPostProcessor for CLOB column:

      curl -X POST -H "Accept:application/json" -H "Content-Type:application/json" http://localhost:8083/connectors -d '
      {
        "name": "source-test-connector",
        "config": {
          "connector.class": "io.debezium.connector.oracle.OracleConnector",
          "tasks.max": "1",
          "database.hostname": "oracle",
          "database.port": "1521",
          "database.user": "c##dbzuser",
          "database.password": "dbz",
          "database.dbname": "orclcdb",
          "database.pdb.name": "orclpdb1",
          "database.connection.adapter": "logminer",
          "topic.prefix": "dbz",
          "schema.name.adjustment.mode": "avro",
          "table.include.list": "C##DBZUSER.TEST_TABLE",
          "include.schema.changes": "false",
          "schema.history.internal.kafka.bootstrap.servers" : "kafka:9092",
          "schema.history.internal.kafka.topic": "schema-changes.test",
          "heartbeat.interval.ms": "60000",
          "log.mining.strategy": "online_catalog",
          "log.mining.query.filter.mode": "in",
          "post.processors": "reselector",
          "reselector.type": "io.debezium.processors.reselect.ReselectColumnsPostProcessor",
          "reselector.reselect.columns.include.list": "C##DBZUSER.TEST_TABLE:DATA",
          "custom.metric.tags": "connector=source-test-connector",
          "transforms": "unwrap",
          "transforms.unwrap.type": "io.debezium.transforms.ExtractNewRecordState",
          "key.converter": "org.apache.kafka.connect.json.JsonConverter",
          "key.converter.schemas.enable": "false",
          "value.converter": "org.apache.kafka.connect.json.JsonConverter",
          "value.converter.schemas.enable": "false"
        }
      }'
      

      3. Insert a new record into the table in a loop:

      INSERT INTO c##dbzuser.test_table (id, data)
      VALUES (1, TO_CLOB('data'));
      

      5. Check Kafka Connect logs:

      AR: java.sql.SQLException: ORA-01000: maximum open cursors exceeded

      Feature request or enhancement

      https://github.com/debezium/debezium/pull/5253

            Unassigned Unassigned
            andrey.pustovetov@gmail.com Andrey Pustovetov
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

              Created:
              Updated:
              Resolved: