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

In case of readonly usage the DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG should not be used

XMLWordPrintable

    • Icon: Bug Bug
    • Resolution: Done
    • Icon: Minor Minor
    • 3.3.0.CR1
    • None
    • oracle-connector
    • None
    • False
    • Hide

      None

      Show
      None
    • False

      What Debezium connector do you use and what version?

      3.2.0.Final

      What is the connector configuration?

      defaultDebeziumProperties.internal.forEach(it::with)
      it.with("database.hostname", defaultDebeziumProperties.database.primaryHostname)
      .with("database.port", defaultDebeziumProperties.database.port)
      .with("database.user", defaultDebeziumProperties.database.user)
      .with("database.password", defaultDebeziumProperties.database.password)
      .with("database.dbname", defaultDebeziumProperties.database.dbname)
      .with("table.include.list", defaultDebeziumProperties.tables)
      .with("log.mining.readonly.hostname", defaultDebeziumProperties.database.hostname) .with("snapshot.mode", "no_data")
      .with("schema.history.internal.store.only.captured.tables.ddl", true)
      .with("schema.history.internal.store.only.captured.databases.ddl", true)
      .with("log.mining.archive.log.only.mode", false)
      .with("log.mining.database.readonly", true)
      .with("log.mining.query.filter.mode", "in")
      .with("event.processing.failure.handling.mode", "warn")
      .with("errors.max.retries", 3)
      .with("snapshot.max.threads", 1)
      .with("log.mining.log.query.max.retries", 5)
      .with("database.query.timeout.ms", 0)
      .with("internal.log.mining.buffer.memory.legacy.transaction.start", true)
      .with("internal.log.mining.read.only", true)
      .with("log.mining.strategy", "dictionary_from_file")
      .with("log.mining.path.dictionary", dictionaryPath)
      .with("log.mining.batch.size.min", minBatchSize.toBigDecimal().toLong())
      .with("log.mining.batch.size.max", maxBatchSize.toBigDecimal().toLong())
      .with("log.mining.batch.size.default", defaultBatchSize.toBigDecimal().toLong())
      .with("log.mining.batch.size.increment", incrementSize.toBigDecimal().toLong())
      .with("query.fetch.size", streamFetchSize.toBigDecimal().toLong())

      What is the captured database version and mode of deployment?

      Oracle OCI

      What behavior do you expect?

      It should start mining in the readonly database

      What behavior do you see?

       

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

      Yes

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

      No

      How to reproduce the issue using our tutorial deployment?

      Start the connector with the configuration added in the card.

       

       

       

      The solution is pretty simple and i already done it and about to submit, just change the get miningOptions to the bellow, already tested with multiple strategy and it's working fine

      // private List<String> getMiningOptions(boolean committedDataOnly) {
          final List<String> miningOptions = new ArrayList<>();
          switch (strategy) {
              case CATALOG_IN_REDO:
                  miningOptions.add("DBMS_LOGMNR.DICT_FROM_REDO_LOGS");
                  miningOptions.add("DBMS_LOGMNR.DDL_DICT_TRACKING");
                  break;
              case DICTIONARY_FROM_FILE:
                  break;
              default:
                  miningOptions.add("DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG");
                  break;
          }
      
          if (useContinuousMining) {
              miningOptions.add("DBMS_LOGMNR.CONTINUOUS_MINE");
          }
      
          if (committedDataOnly) {
              miningOptions.add("DBMS_LOGMNR.COMMITTED_DATA_ONLY");
          }
      
          miningOptions.add("DBMS_LOGMNR.NO_ROWID_IN_STMT");
      
          return miningOptions;
      }

              Unassigned Unassigned
              lucasgazire Lucas Gazire
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

                Created:
                Updated:
                Resolved: