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

Single quote replication includes escaped quotes for N(CHAR/VARCHAR) columns

XMLWordPrintable

    • Important

      What Debezium connector do you use and what version?

      Debezium version: 2.4.0-SNAPSHOT (c748fc494b4d1bd6acaf4fe2d7d885cb5562d2f3)
      Using debezium-server: 2.4.0-SNAPSHOT (e9365d39e76c45f1c42855675c72aab8b0a7ee1d)

      What is the connector configuration?

      debezium.source.topic.prefix=yb-voyager
      debezium.source.database.server.name=yb-voyager
      
      debezium.source.database.url=jdbc:oracle:thin:@(DESCRIPTION=(ADDRESS=(PROTOCOL=TCP)(HOST=127.0.0.1)(PORT=1521))(CONNECT_DATA=(SERVICE_NAME=ORCLCDB)))
      debezium.source.connector.class=io.debezium.connector.oracle.OracleConnector
      debezium.source.database.dbname=PLACEHOLDER
      debezium.source.schema.include.list=TEST_SCHEMA
      debezium.source.hstore.handling.mode=map
      debezium.source.database.history=io.debezium.relational.history.FileDatabaseHistory
      debezium.source.database.history.file.filename=data/history.dat
      debezium.source.schema.history.internal=io.debezium.storage.file.history.FileSchemaHistory
      debezium.source.schema.history.internal.file.filename=data/schema_history.json
      debezium.source.schema.history.internal.skip.unparseable.ddl=true
      debezium.source.schema.history.internal.store.only.captured.tables.ddl=true
      debezium.source.schema.history.internal.store.only.captured.databases.ddl=true
      debezium.source.include.schema.changes=false
      debezium.source.log.mining.batch.size.min=10000
      debezium.source.log.mining.batch.size.max=100000
      debezium.source.log.mining.batch.size.default=10000
      debezium.source.log.mining.query.filter.mode=in
      debezium.source.log.mining.sleep.time.default.ms=200
      debezium.source.log.mining.sleep.time.max.ms=400
      debezium.source.max.batch.size=10000
      debezium.source.max.queue.size=50000
      debezium.source.query.fetch.size=10000
      
      debezium.sink.type=http
      debezium.sink.http.url=https://rbaskets.in/web/dbzm-oracle-reproducer
      
      debezium.source.internal.log.mining.transaction.snapshot.boundary.mode=all
      
      debezium.source.database.pdb.name=ORCLPDB1
      
      debezium.source.column.include.list=TEST_SCHEMA.TEST_NCHAR.* 

      What is the captured database version and mode of depoyment?

      on-prem oracle CDB (in docker)

      What behaviour do you expect?

      values don't have extra single quotes that were introduced for escaping

      What behaviour do you see?

      values have extra single quotes

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

      yes

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

      How to reproduce the issue using our tutorial deployment?

      Table at snapshot:

      Columns
      NAME         DATA TYPE      NULL  DEFAULT    COMMENTS
      *ID          NUMBER(38,0)   No
       VAL         NCHAR(10)      Yes
      
      SQL> select * from test_nchar;
      
         ID VAL
      _____ _____________
          1 a'b
          2 a'b\c 

      Table after streaming:

      SQL> select * from test_nchar;
      
         ID VAL
      _____ _____________
          1 a'b
          2 a'b\c
          3 a'b
          4 a'b\c 

      event for id=2(snapshot):

       "payload": {
          "before": null,
          "after": {
            "ID": "Ag==",
            "VAL": "a'b\\c     "
          },
          "source": {
            "version": "2.4.0-SNAPSHOT",
            "connector": "oracle",
            "name": "yb-voyager",
            "ts_ms": 1695374963000,
            "snapshot": "last",
            "db": "ORCLPDB1",
            "sequence": null,
            "schema": "TEST_SCHEMA",
            "table": "TEST_NCHAR",
            "txId": null,
            "scn": "16477528",
            "commit_scn": null,
            "lcr_position": null,
            "rs_id": null,
            "ssn": 0,
            "redo_thread": null,
            "user_name": null
          },
          "op": "r",
          "ts_ms": 1695374973278,
          "transaction": null
        } 

      event for id=4(streaming):

      "payload": {
          "before": null,
          "after": {
            "ID": "BA==",
            "VAL": "a''b\\c     "
          },
          "source": {
            "version": "2.4.0-SNAPSHOT",
            "connector": "oracle",
            "name": "yb-voyager",
            "ts_ms": 1695375002000,
            "snapshot": "false",
            "db": "ORCLPDB1",
            "sequence": null,
            "schema": "TEST_SCHEMA",
            "table": "TEST_NCHAR",
            "txId": "03000800ac040000",
            "scn": "16510785",
            "commit_scn": "16510789",
            "lcr_position": null,
            "rs_id": "0x0001a0.00043126.01b8",
            "ssn": 0,
            "redo_thread": 1,
            "user_name": "TEST_SCHEMA"
          },
          "op": "c",
          "ts_ms": 1695375024365,
          "transaction": null
        } 

      As you can see, the VAL column has an extra quote in the event for id=4(streaming), i.e. the single quote is escaped. the web link for the requests payload - https://rbaskets.in/web/dbzm-oracle-reproducer (token: G-vV0mTOx2s0XfRUcUWLIAEnq7nt-Uatqh8KD0zqIoZ5)

      Did a little debugging; I believe the issue is because oracle produces the dml statement with the function UNISTR(val) for NCHAR columns, it ends up in this block in the LogminerDmlParser, which basically doesn't use the collectedValue which is unescaped, but instead uses it as-is.

       

      Zulip link: https://debezium.zulipchat.com/#narrow/stream/302529-community-general/topic/Single.20quote.20replication/near/392483627

              ccranfor@redhat.com Chris Cranford
              makalaaneesh Aneesh Makala (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

                Created:
                Updated:
                Resolved: