• Icon: Bug Bug
    • Resolution: Done
    • Icon: Major Major
    • 1.9.0.Final
    • None
    • oracle-connector
    • None
    • False
    • None
    • False

      Hello!
       
      I have found the bug in single quote repilacion '.
      When there is update or insert event then:
      The value in oracle=Test'quote


      Value in kafka topic = Test''quote

      {"schema":\{"type":"struct","fields":[{"type":"string","optional":false,"field":"ID"}],"optional":false,"name":"dev_bodb.MZIGISOVS.QUOTE_TEST.Key"},"payload":\{"ID":"2"}} \{"schema":{"type":"struct","fields":[{"type":"string","optional":true,"field":"QUOTE_FIELD"},\{"type":"string","optional":false,"field":"ID"}],"optional":false,"name":"dev_bodb.MZIGISOVS.QUOTE_TEST.Value"},"payload":\{"QUOTE_FIELD":"test''quote","ID":"2"}}

       
      If there is initial snapshot:
      The value in oracle=test'quote

      Value in kafka topic = test'quote

      {"schema":\{"type":"struct","fields":[{"type":"string","optional":false,"field":"ID"}],"optional":false,"name":"dev_bodb.MZIGISOVS.QUOTE_TEST.Key"},"payload":\{"ID":"2"}} \{"schema":{"type":"struct","fields":[{"type":"string","optional":true,"field":"QUOTE_FIELD"},\{"type":"string","optional":false,"field":"ID"}],"optional":false,"name":"dev_bodb.MZIGISOVS.QUOTE_TEST.Value"},"payload":\{"QUOTE_FIELD":"test'quote","ID":"2"}}

      I have made the debug of this logminerParser and found that parser reads the value from oracle redo log and make mistake in parser, as I correctly understand perhaps the problem is here, could you take look:

      debezium-connector-oracle\src\main\java\io\debezium\connector\oracle\logminer\parser\LogMinerDmlParser.java -> 363 row

      How to reproduce:

      Create table: 

      CREATE TABLE QUOTE_TEST (
      QUOTE_FIELD VARCHAR(20),
      ID NUMBER,
      PRIMARY KEY(ID)
      );
      INSERT INTO QUOTE_TEST (QUOTE_FIELD,ID)
      VALUES ('test''quote',1);

      Create config.properties: 

      name=oracle-source-quote-test
      connector.class=io.debezium.connector.oracle.OracleConnector
      database.hostname=
      database.port=
      database.user=
      database.password=
      database.dbname=
      database.server.name=
      tasks.max=1
      heartbeat.interval.ms=60000
      database.history.kafka.bootstrap.servers=localhost:9092
      database.history.kafka.topic=schema-changes.inventory
      database.history.skip.unparseable.ddl=true
      database.history.store.only.captured.tables.ddl=true
      schema.include.list={{yourShema}}
      table.include.list={{yourShema}}.QUOTE_TEST
      column.include.list={{yourShema}}.QUOTE_TEST.QUOTE_FIELD,{{yourShema}}.QUOTE_TEST.ID
      database.connection.adapter=logminer
      decimal.handling.mode=string
      log.mining.strategy=online_catalog
      event.processing.failure.handling.mode=skip
      time.precision.mode=connect
      include.schema.changes=false
      
      transforms=unwrap,route
      
      transforms.unwrap.type=io.debezium.transforms.ExtractNewRecordState
      transforms.unwrap.drop.tombstones=false
      transforms.unwrap.delete.handling.mode=none
      
      transforms.route.type=org.apache.kafka.connect.transforms.RegexRouter
      transforms.route.regex=([^.]+)\\.([^.]+)\\.([^.]+)
      transforms.route.replacement=$3

      Start zookeeper/kafka/debezium

      Wait for initial snapshot and check value of QUOTE_TEST.QUOTE_FIELD.

      make update or insert:

      update {{yourSchema}}.QUOTE_TEST set QUOTE_FIELD = 'test_quo''te' where id = 1;

      check value of QUOTE_TEST.QUOTE_FIELD.

      Thank you!

            [DBZ-4891] Single quotes replication

            if you are still having problems, please raise a new Jira with the use case, all the details that surround how to reproduce it and we'll take a look.
            In the new Jira, can you also reference back to this Jira as well; thanks.

            Chris Cranford added a comment - if you are still having problems, please raise a new Jira with the use case, all the details that surround how to reproduce it and we'll take a look. In the new Jira, can you also reference back to this Jira as well; thanks.

            Hi, I'm still having this issue with 1.9.2.Final, but looking at these comments I'm not sure if it has been fixed, can anyone confirm ?

            Matthias Nantier (Inactive) added a comment - Hi, I'm still having this issue with 1.9.2.Final, but looking at these comments I'm not sure if it has been fixed, can anyone confirm ?

            Hi quy196hp, I've logged DBZ-4994, I'll take a look.

            Chris Cranford added a comment - Hi quy196hp , I've logged DBZ-4994 , I'll take a look.

             

            Caused by: io.debezium.text.ParsingException: Failed to parse operator at index 221: DECLARE 
             loc_c CLOB; 
             buf_c VARCHAR2(6396); 
             loc_b BLOB; 
             buf_b RAW(6396); 
             loc_nc NCLOB; 
             buf_nc NVARCHAR2(6396); 
            BEGIN
             select "COLUMN2" into loc_c from "VNSOCIAL_DATA"."Z_TEST_CDC_CLOB" where "COLUMN1" = '2"''" sd f""" '''''''' ''''' for update;
                at io.debezium.connector.oracle.logminer.parser.SelectLobParser.parseOperator(SelectLobParser.java:235)
                at io.debezium.connector.oracle.logminer.parser.SelectLobParser.parseWhere(SelectLobParser.java:192)
                at io.debezium.connector.oracle.logminer.parser.SelectLobParser.parse(SelectLobParser.java:73)
                ... 16 more
            
            

            I try with string as

             2"''" sd f""" '''''''' ''''

            And it raises error.

            Qu�� ����� (Inactive) added a comment -   Caused by: io.debezium.text.ParsingException: Failed to parse operator at index 221: DECLARE   loc_c CLOB;   buf_c VARCHAR2(6396);   loc_b BLOB;   buf_b RAW(6396);   loc_nc NCLOB;   buf_nc NVARCHAR2(6396);  BEGIN  select "COLUMN2" into loc_c from "VNSOCIAL_DATA" . "Z_TEST_CDC_CLOB" where "COLUMN1" = '2 "' '" sd f""" ' ''''''' ' '''' for update;     at io.debezium.connector.oracle.logminer.parser.SelectLobParser.parseOperator(SelectLobParser.java:235)     at io.debezium.connector.oracle.logminer.parser.SelectLobParser.parseWhere(SelectLobParser.java:192)     at io.debezium.connector.oracle.logminer.parser.SelectLobParser.parse(SelectLobParser.java:73)     ... 16 more I try with string as 2 " ''" sd f""" ' ''''''' ' ''' And it raises error.

            quy196hp Could you please share a sample DDL and DML so we can try to reproduce?

            Jiri Pechanec added a comment - quy196hp Could you please share a sample DDL and DML so we can try to reproduce?

            I tried version 1.9 final but the error still seems to be there. Can someone confirm this has been fixed? 

            Qu�� ����� (Inactive) added a comment - I tried version 1.9 final but the error still seems to be there. Can someone confirm this has been fixed? 

            Released

            Chris Cranford added a comment - Released

            Pull request: https://github.com/debezium/debezium/pull/3341

             

            Reworked the code after getting the feedback on the initial implementation.

            The performance of the new implementation was compared to the current one. The test not very good but gives the idea of the impact.
            For the existing and the new implementations the same test was repeated for 10 times. Each time there was 10000000 method calls. In the printout below you will find the total time in each repetition. Tried the with different JREs with very very different results, probably, due to internal optimizator. So I just decided to put here the figures I have, don't have time for proper investigation

            Coretto 8

            parseWhereClauseOri:2428:2413:2363:2356:2359:2376:2443:2372:2359:2415
            parseWhereClauseNew:2087:2067:2086:2031:2063:2042:2117:2145:2115:2237
            --------------------------------------------------------------------
            parseSetClauseOri:3879:3912:3970:3870:3841:3868:3947:3831:3837:3778
            parseSetClauseNew:3179:3157:3143:3161:3163:3132:3147:3229:3151:3179
            --------------------------------------------------------------------
            parseColumnValuesClauseOri: 960: 953: 958: 980: 970: 960: 968: 973: 950: 953
            parseColumnValuesClauseNew:1202:1224:1207:1209:1217:1192:1217:1241:1181:1200
            --------------------------------------------------------------------
            

            Coretto 11

            parseWhereClauseOri:2204:2238:2443:2493:2305:2154:2324:2517:2210:2496
            parseWhereClauseNew:3035:2636:2838:3084:3027:2653:2764:3070:2867:2623
            --------------------------------------------------------------------
            parseSetClauseOri:3506:3679:3562:3905:3671:3611:3603:3615:4076:4215
            parseSetClauseNew:5452:5676:5281:5335:5492:5295:5260:5461:5741:7851
            --------------------------------------------------------------------
            parseColumnValuesClauseOri:1770:1511:1544:1408:1455:1397:1565:1507:1451:1378
            parseColumnValuesClauseNew:2056:2032:1889:1828:1683:1671:1602:1405:1380:1467
            --------------------------------------------------------------------
            

            Aleksejs Sibanovs (Inactive) added a comment - Pull request: https://github.com/debezium/debezium/pull/3341   Reworked the code after getting the feedback on the initial implementation. The performance of the new implementation was compared to the current one. The test not very good but gives the idea of the impact. For the existing and the new implementations the same test was repeated for 10 times. Each time there was 10000000 method calls. In the printout below you will find the total time in each repetition. Tried the with different JREs with very very different results, probably, due to internal optimizator. So I just decided to put here the figures I have, don't have time for proper investigation Coretto 8 parseWhereClauseOri:2428:2413:2363:2356:2359:2376:2443:2372:2359:2415 parseWhereClauseNew:2087:2067:2086:2031:2063:2042:2117:2145:2115:2237 -------------------------------------------------------------------- parseSetClauseOri:3879:3912:3970:3870:3841:3868:3947:3831:3837:3778 parseSetClauseNew:3179:3157:3143:3161:3163:3132:3147:3229:3151:3179 -------------------------------------------------------------------- parseColumnValuesClauseOri: 960: 953: 958: 980: 970: 960: 968: 973: 950: 953 parseColumnValuesClauseNew:1202:1224:1207:1209:1217:1192:1217:1241:1181:1200 -------------------------------------------------------------------- Coretto 11 parseWhereClauseOri:2204:2238:2443:2493:2305:2154:2324:2517:2210:2496 parseWhereClauseNew:3035:2636:2838:3084:3027:2653:2764:3070:2867:2623 -------------------------------------------------------------------- parseSetClauseOri:3506:3679:3562:3905:3671:3611:3603:3615:4076:4215 parseSetClauseNew:5452:5676:5281:5335:5492:5295:5260:5461:5741:7851 -------------------------------------------------------------------- parseColumnValuesClauseOri:1770:1511:1544:1408:1455:1397:1565:1507:1451:1378 parseColumnValuesClauseNew:2056:2032:1889:1828:1683:1671:1602:1405:1380:1467 --------------------------------------------------------------------

            Unfortunately, I do not see the field to add the link to Pull request, but here it is:
            https://github.com/debezium/debezium/pull/3335

            In DML parses logic changed for Inserts values, and Where clause (affects updates and deletes).
            Oracle has special single quote handling, as string values goes insige single quotes (do not confuse with double quotes)

            a single quote '
            Oracle escaped single quote: ''
            Double quote : "
            

            In the current Debezium version, initial snapshot works correctly, passing value "Bob's dog" into Kafka as "Bob's dog". But if you are using logminer, then the following statement will be handled incorrectly:

            update table set column1='Bob''s dog' where column_pk=1
            

            It will cause passing column1 new value = Bob''s dog (escaping signle quote is not stripped)
            Same applies to all DML functions.

            Suggested changes fixes this behaviour. During the iteration over the single quoted values, all the escaped single quotes positions are stored in the stack and after the value is ready, they are removed from the result.

            Aleksejs Sibanovs (Inactive) added a comment - Unfortunately, I do not see the field to add the link to Pull request, but here it is: https://github.com/debezium/debezium/pull/3335 In DML parses logic changed for Inserts values, and Where clause (affects updates and deletes). Oracle has special single quote handling, as string values goes insige single quotes (do not confuse with double quotes) a single quote ' Oracle escaped single quote: '' Double quote : " In the current Debezium version, initial snapshot works correctly, passing value "Bob's dog" into Kafka as "Bob's dog". But if you are using logminer, then the following statement will be handled incorrectly: update table set column1= 'Bob' 's dog' where column_pk=1 It will cause passing column1 new value = Bob''s dog (escaping signle quote is not stripped) Same applies to all DML functions. Suggested changes fixes this behaviour. During the iteration over the single quoted values, all the escaped single quotes positions are stored in the stack and after the value is ready, they are removed from the result.

            During the work on the fix, found out that there are issues with escaped signle quotes not only in updates, but also in other DMLs.

            Found DBZ-3413 which has tests checking that the values contains still escaped quotes. It means that in the Kafka queue the single quote will be escaped, which is wrong as far as I see it. Therefore pull request will contain also fixes for the unit test marked as DBZ-3413

             

            Aleksejs Sibanovs (Inactive) added a comment - During the work on the fix, found out that there are issues with escaped signle quotes not only in updates , but also in other DMLs. Found DBZ-3413 which has tests checking that the values contains still escaped quotes. It means that in the Kafka queue the single quote will be escaped, which is wrong as far as I see it. Therefore pull request will contain also fixes for the unit test marked as DBZ-3413  

              Unassigned Unassigned
              mzigisov maksim zigisov (Inactive)
              Votes:
              1 Vote for this issue
              Watchers:
              7 Start watching this issue

                Created:
                Updated:
                Resolved: