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

During step 5, the connector scans all tables regardless of whitelist / blacklist filtering. For schemas with a large number of tables in proportion to the whitelist / blacklist exclusions can result in significant wait times (approximately 2 sec / table)

    XMLWordPrintable

Details

    • Bug
    • Resolution: Obsolete
    • Major
    • None
    • 1.1.0.Final
    • oracle-connector
    • None
    • Hide

      Create a schema in an Oracle database and populate it with over 34000 tables:

      Create an Apache Kafka connector with the following connector config:

      { "name": "inventory-connector", "config":

      { "connector.class": "io.debezium.connector.oracle.OracleConnector", "tasks.max": "1", "table.whitelist":"(.)CUSTOMERS,(.)PRODUCTS", "database.server.name": "inventory", "database.hostname": "localhost", "database.port": "1521", "database.user": "c##xstrm", "database.password": "xs", "database.dbname": "ORCLCDB", "database.pdb.name": "ORCLPDB1", "database.out.server.name": "dbzxout", "snapshot.delay.ms": "5000", "database.history.kafka.bootstrap.servers": "localhost:9092", "database.history.kafka.topic": "schema-changes.inventory" }

      }

      receive the the following log file:

      [2020-04-09 05:48:01,809] INFO Metrics registered (io.debezium.pipeline.ChangeEventSourceCoordinator:82)
      [2020-04-09 05:48:01,809] INFO Context created (io.debezium.pipeline.ChangeEventSourceCoordinator:85)
      [2020-04-09 05:48:03,026] INFO Snapshot step 1 - Preparing (io.debezium.relational.RelationalSnapshotChangeEventSource:94)
      [2020-04-09 05:48:03,027] INFO Snapshot step 2 - Determining captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource:105)
      [2020-04-09 05:48:11,984] INFO Snapshot step 3 - Locking captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource:112)
      [2020-04-09 05:48:12,113] INFO Snapshot step 4 - Determining snapshot offset (io.debezium.relational.RelationalSnapshotChangeEventSource:118)
      [2020-04-09 05:48:12,619] INFO Snapshot step 5 - Reading structure of captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource:121)

      ... A lot of lines removed and 10.25 hours later ...

      [2020-04-09 16:09:23,240] INFO Snapshot step 6 - Persisting schema history (io.debezium.relational.RelationalSnapshotChangeEventSource:125)
      [2020-04-09 16:09:48,752] INFO Snapshot step 7 - Snapshotting data (io.debezium.relational.RelationalSnapshotChangeEventSource:137)
      [2020-04-09 16:09:48,753] INFO Exporting data from table '...Some Table...' (io.debezium.relational.RelationalSnapshotChangeEventSource:280)
      [2020-04-09 16:09:48,753] INFO For table '...Some Table...' using select statement: 'SELECT * FROM ...Some Table... AS OF SCN 9999999999' (io.debezium.relational.Rel ationalSnapshotChangeEventSource:287)
      [2020-04-09 16:09:49,618] INFO Finished exporting 23 records for table '...Some Table...'; total duration '00:00:00.864' (io.debezium.relational.RelationalSnapshotChangeEventSource :330)
      [2020-04-09 16:09:49,632] INFO Snapshot - Final stage (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource:79)
      [2020-04-09 16:09:50,164] INFO Snapshot ended with SnapshotResult [status=COMPLETED, offset=OracleOffsetContext [scn=10784643213]] (io.debezium.pipeline.ChangeEventSourceCoordinator:90)
      [2020-04-09 16:09:50,167] INFO Connected metrics set to 'true' (io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics:59)
      [2020-04-09 16:09:50,167] INFO Starting streaming (io.debezium.pipeline.ChangeEventSourceCoordinator:100)

      Note in the oracle database that a query that is a UNION of two selects: One on all_tables and the other a join between all_indexes and all_ind_columns

      Show
      Create a schema in an Oracle database and populate it with over 34000 tables: Create an Apache Kafka connector with the following connector config: { "name": "inventory-connector", "config": { "connector.class": "io.debezium.connector.oracle.OracleConnector", "tasks.max": "1", "table.whitelist":"(.)CUSTOMERS,(.)PRODUCTS", "database.server.name": "inventory", "database.hostname": "localhost", "database.port": "1521", "database.user": "c##xstrm", "database.password": "xs", "database.dbname": "ORCLCDB", "database.pdb.name": "ORCLPDB1", "database.out.server.name": "dbzxout", "snapshot.delay.ms": "5000", "database.history.kafka.bootstrap.servers": "localhost:9092", "database.history.kafka.topic": "schema-changes.inventory" } } receive the the following log file: [2020-04-09 05:48:01,809] INFO Metrics registered (io.debezium.pipeline.ChangeEventSourceCoordinator:82) [2020-04-09 05:48:01,809] INFO Context created (io.debezium.pipeline.ChangeEventSourceCoordinator:85) [2020-04-09 05:48:03,026] INFO Snapshot step 1 - Preparing (io.debezium.relational.RelationalSnapshotChangeEventSource:94) [2020-04-09 05:48:03,027] INFO Snapshot step 2 - Determining captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource:105) [2020-04-09 05:48:11,984] INFO Snapshot step 3 - Locking captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource:112) [2020-04-09 05:48:12,113] INFO Snapshot step 4 - Determining snapshot offset (io.debezium.relational.RelationalSnapshotChangeEventSource:118) [2020-04-09 05:48:12,619] INFO Snapshot step 5 - Reading structure of captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource:121) ... A lot of lines removed and 10.25 hours later ... [2020-04-09 16:09:23,240] INFO Snapshot step 6 - Persisting schema history (io.debezium.relational.RelationalSnapshotChangeEventSource:125) [2020-04-09 16:09:48,752] INFO Snapshot step 7 - Snapshotting data (io.debezium.relational.RelationalSnapshotChangeEventSource:137) [2020-04-09 16:09:48,753] INFO Exporting data from table '...Some Table...' (io.debezium.relational.RelationalSnapshotChangeEventSource:280) [2020-04-09 16:09:48,753] INFO For table '...Some Table...' using select statement: 'SELECT * FROM ...Some Table... AS OF SCN 9999999999' (io.debezium.relational.Rel ationalSnapshotChangeEventSource:287) [2020-04-09 16:09:49,618] INFO Finished exporting 23 records for table '...Some Table...'; total duration '00:00:00.864' (io.debezium.relational.RelationalSnapshotChangeEventSource :330) [2020-04-09 16:09:49,632] INFO Snapshot - Final stage (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource:79) [2020-04-09 16:09:50,164] INFO Snapshot ended with SnapshotResult [status=COMPLETED, offset=OracleOffsetContext [scn=10784643213] ] (io.debezium.pipeline.ChangeEventSourceCoordinator:90) [2020-04-09 16:09:50,167] INFO Connected metrics set to 'true' (io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics:59) [2020-04-09 16:09:50,167] INFO Starting streaming (io.debezium.pipeline.ChangeEventSourceCoordinator:100) Note in the oracle database that a query that is a UNION of two selects: One on all_tables and the other a join between all_indexes and all_ind_columns

    Description

      I'm using the Debezium Connector for Golden Gate 1.2 version and the Apache Kafka Confluent Enterprise 5.3.1. My schema has more than 34000 tables and I'm only interested in < 200 of them. I've whitelisted the tables that I want; however, when create a new connector step 5 takes more than 10 hours to complete. I reviewed the database logs while this occurs and identified that a query is cycling through all_tables & all_index / all_ind_columns regardless of whether the tables are in the whitelist.

      I found @ the following source location:

      https://github.com/debezium/debezium-incubator/blob/e84db3030b5c7de635248e3fe1146c7f35165cb0/debezium-connector-oracle/src/main/java/io/debezium/connector/oracle/OracleSnapshotChangeEventSource.java

      190 - 206 specifies that the developer believes that it's more efficient to ignore the filter for the schema and filter the result set row by row. On 198, he calls readSchema

      https://github.com/debezium/debezium-incubator/blob/e84db3030b5c7de635248e3fe1146c7f35165cb0/debezium-connector-oracle/src/main/java/io/debezium/connector/oracle/OracleConnection.java

      100-146 specifies readSchema, where on line 104, the developer does not provide a filter argument for the Table - even though he has it (see tableFilter variable)

      https://github.com/debezium/debezium/blob/master/debezium-core/src/main/java/io/debezium/jdbc/JdbcConnection.java

      1028 - 1087 specifies the super.readSchema, which also ignores the tableFilter on line on 1049.

      Recommend Actions:

      1. Add a conditional that checks the proportion of excluded tables with that in the schema. If the number is less than 10%, use the exclude the tables as a part of the query; otherwise perform the processing as is.

      2. Review why the query can't use a UNION ALL instead of UNION as this is the primary reason why the SQL statement takes too long.

      Attachments

        Issue Links

          Activity

            People

              ccranfor@redhat.com Chris Cranford
              jwwallac Jonathan Wallace (Inactive)
              Votes:
              1 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: