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

datatype conversion failure for numeric primarykey columns

    XMLWordPrintable

Details

    • Bug
    • Resolution: Done
    • Major
    • 1.4.1.Final
    • 1.4.0.Alpha1
    • oracle-connector
    • None
    • False
    • False
    • Undefined
    • Hide

      1. Create a table:

      CREATE TABLE contacts
        (
          contact_id NUMBER 
                     GENERATED BY DEFAULT AS IDENTITY START WITH 320 
                     PRIMARY KEY,
          first_name  VARCHAR2( 255 ) NOT NULL,
          last_name   VARCHAR2( 255 ) NOT NULL,
          email       VARCHAR2( 255 ) NOT NULL,
          phone       VARCHAR2( 20 )          ,
          customer_id NUMBER                  ,
          CONSTRAINT fk_contacts_customers 
            FOREIGN KEY( customer_id )
            REFERENCES customers( customer_id ) 
            ON DELETE CASCADE
        );
      

      2. Insert few records

      ALTER TABLE contacts DISABLE CONSTRAINT fk_contacts_customers;
      SET DEFINE OFF;
      Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (1,'Flor','Stone','flor.stone@raytheon.com','+1 317 123 4104',1);
      Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (2,'Lavera','Emerson','lavera.emerson@plainsallamerican.com','+1 317 123 4111',2);
      Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (3,'Fern','Head','fern.head@usfoods.com','+1 812 123 4115',3);
      Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (4,'Shyla','Ortiz','shyla.ortiz@abbvie.com','+1 317 123 4126',4);
      Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (5,'Jeni','Levy','jeni.levy@centene.com','+1 812 123 4129',5);
      Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (6,'Matthias','Hannah','matthias.hannah@chs.net','+1 219 123 4136',6);
      

      3. Start debezium with below configurstions:

      {
          "name": "LOGON-CONNECTOR",
          "config": {
              "connector.class" : "io.debezium.connector.oracle.OracleConnector",
              "tasks.max" : "1",
              "database.server.name" : "LQ1",
              "database.hostname" : "xxxxx.us-east-1.rds.amazonaws.com",
              "database.out.server.name" : "dbzxout",
              "database.port" : "1521",
              "database.user" : "clogminer",
              "database.password" : "xxxxxxx",
              "database.dbname" : "XXX",
              "database.schema" : "ADMIN",
              "database.history.kafka.bootstrap.servers" : "kafka:9092",
              "table.include.list": "ADMIN.CONTACTS",
              "database.history.kafka.topic": "schema-changes.logon",
              "database.connection.adapter": "logminer",
              "log.mining.strategy": "online_catalog",
              "key.converter": "io.confluent.connect.avro.AvroConverter",
              "key.converter.schema.registry.url": "http://schema-registry:8081",
              "value.converter": "io.confluent.connect.avro.AvroConverter",
              "value.converter.schema.registry.url": "http://schema-registry:8081"   
          }
      }
      

      4. Insert or Update the table

      Show
      1. Create a table: CREATE TABLE contacts ( contact_id NUMBER GENERATED BY DEFAULT AS IDENTITY START WITH 320 PRIMARY KEY, first_name VARCHAR2( 255 ) NOT NULL, last_name VARCHAR2( 255 ) NOT NULL, email VARCHAR2( 255 ) NOT NULL, phone VARCHAR2( 20 ) , customer_id NUMBER , CONSTRAINT fk_contacts_customers FOREIGN KEY( customer_id ) REFERENCES customers( customer_id ) ON DELETE CASCADE ); 2. Insert few records ALTER TABLE contacts DISABLE CONSTRAINT fk_contacts_customers; SET DEFINE OFF; Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (1, 'Flor' , 'Stone' , 'flor.stone@raytheon.com' , '+1 317 123 4104' ,1); Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (2, 'Lavera' , 'Emerson' , 'lavera.emerson@plainsallamerican.com' , '+1 317 123 4111' ,2); Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (3, 'Fern' , 'Head' , 'fern.head@usfoods.com' , '+1 812 123 4115' ,3); Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (4, 'Shyla' , 'Ortiz' , 'shyla.ortiz@abbvie.com' , '+1 317 123 4126' ,4); Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (5, 'Jeni' , 'Levy' , 'jeni.levy@centene.com' , '+1 812 123 4129' ,5); Insert into CONTACTS (CONTACT_ID,FIRST_NAME,LAST_NAME,EMAIL,PHONE,CUSTOMER_ID) values (6, 'Matthias' , 'Hannah' , 'matthias.hannah@chs.net' , '+1 219 123 4136' ,6); 3. Start debezium with below configurstions: { "name" : "LOGON-CONNECTOR" , "config" : { "connector.class" : "io.debezium.connector.oracle.OracleConnector" , "tasks.max" : "1" , "database.server.name" : "LQ1" , "database.hostname" : "xxxxx.us-east-1.rds.amazonaws.com" , "database.out.server.name" : "dbzxout" , "database.port" : "1521" , "database.user" : "clogminer" , "database.password" : "xxxxxxx" , "database.dbname" : "XXX" , "database.schema" : "ADMIN" , "database.history.kafka.bootstrap.servers" : "kafka:9092" , "table.include.list" : "ADMIN.CONTACTS" , "database.history.kafka.topic" : "schema-changes.logon" , "database.connection.adapter" : "logminer" , "log.mining.strategy" : "online_catalog" , "key.converter" : "io.confluent.connect.avro.AvroConverter" , "key.converter.schema.registry.url" : "http: //schema-registry:8081" , "value.converter" : "io.confluent.connect.avro.AvroConverter" , "value.converter.schema.registry.url" : "http: //schema-registry:8081" } } 4. Insert or Update the table

    Description

      When an update/insert is done on a table with a numeric primarykey, debezium fails with errorĀ 

      java.lang.IllegalArgumentException: Unexpected value for JDBC type 2 and column CUSTOMER_ID NUMBER(0) NOT NULL: class=class org.apache.kafka.connect.data.Struct
      

      But this does not happen when the initial logs are read. Only during subsequent new updates and inserts. Also tables without a primarkey defined does not have this issue, they work perfectly as expected. See below complete stack trace. I think the issue is that the Number datatype was not defined with a default precision. I tried adding a precision to the DDL for the key and it was Ok.

      2020-11-09 11:59:18,396 ERROR || Producer failure [io.debezium.pipeline.ErrorHandler]
      org.apache.kafka.connect.errors.ConnectException: Error while processing event at offset {commit_scn=1410724, transaction_id=null, scn=1410720}
      at io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:220)
      at io.debezium.connector.oracle.logminer.LogMinerQueryResultProcessor.lambda$processResult$0(LogMinerQueryResultProcessor.java:196)
      at io.debezium.connector.oracle.logminer.TransactionalBuffer.lambda$commit$1(TransactionalBuffer.java:204)
      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:834)
      Caused by: java.lang.IllegalArgumentException: Unexpected value for JDBC type 2 and column CONTACT_ID NUMBER(0) NOT NULL: class=class org.apache.kafka.connect.data.Struct
      at io.debezium.jdbc.JdbcValueConverters.handleUnknownData(JdbcValueConverters.java:1247)
      at io.debezium.jdbc.JdbcValueConverters.convertValue(JdbcValueConverters.java:1290)
      at io.debezium.jdbc.JdbcValueConverters.toBigDecimal(JdbcValueConverters.java:1024)
      at io.debezium.jdbc.JdbcValueConverters.convertDecimal(JdbcValueConverters.java:1016)
      at io.debezium.connector.oracle.OracleValueConverters.convertDecimal(OracleValueConverters.java:292)
      at io.debezium.connector.oracle.OracleValueConverters.convertNumeric(OracleValueConverters.java:297)
      at io.debezium.connector.oracle.OracleValueConverters.convertVariableScale(OracleValueConverters.java:390)
      at io.debezium.connector.oracle.OracleValueConverters.lambda$getNumericConverter$13(OracleValueConverters.java:203)
      at io.debezium.relational.TableSchemaBuilder.lambda$createKeyGenerator$3(TableSchemaBuilder.java:191)
      at io.debezium.relational.TableSchema.keyFromColumnData(TableSchema.java:130)
      at io.debezium.relational.RelationalChangeRecordEmitter.emitCreateRecord(RelationalChangeRecordEmitter.java:65)
      at io.debezium.relational.RelationalChangeRecordEmitter.emitChangeRecords(RelationalChangeRecordEmitter.java:45)
      at io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:193)
      ... 5 more
      2020-11-09 11:59:18,426 INFO || WorkerSourceTask{id=LOGON-CONNECTOR-0} Committing offsets [org.apache.kafka.connect.runtime.WorkerSourceTask]
      2020-11-09 11:59:18,426 INFO || WorkerSourceTask{id=LOGON-CONNECTOR-0} flushing 0 outstanding messages for offset commit [org.apache.kafka.connect.runtime.WorkerSourceTask]
      2020-11-09 11:59:18,426 ERROR || WorkerSourceTask{id=LOGON-CONNECTOR-0} Task threw an uncaught and unrecoverable exception [org.apache.kafka.connect.runtime.WorkerTask]
      org.apache.kafk.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
      at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:42)
      at io.debezium.connector.oracle.logminer.TransactionalBuffer.lambda$commit$1(TransactionalBuffer.java:214)
      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:834)
      Caused by: org.apache.kafka.connect.errors.ConnectException: Error while processing event at offset {commit_scn=1410724, transaction_id=null, scn=1410720}
      at io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:220)
      at io.debezium.connector.oracle.logminer.LogMinerQueryResultProcessor.lambda$processResult$0(LogMinerQueryResultProcessor.java:196)
      at io.debezium.connector.oracle.logminer.TransactionalBuffer.lambda$commit$1(TransactionalBuffer.java:204)
      ... 3 more
      Caused by: java.lang.IllegalArgumentException: Unexpected value for JDBC type 2 and column CONTACT_ID NUMBER(0) NOT NULL: class=class org.apache.kafka.connect.data.Struct
      at io.debezium.jdbc.JdbcValueConverters.handleUnknownData(JdbcValueConverters.java:1247)
      at io.debezium.jdbc.JdbcValueConverters.convertValue(JdbcValueConverters.java:1290)
      at io.debezium.jdbc.JdbcValueConverters.toBigDecimal(JdbcValueConverters.java:1024)
      at io.debezium.jdbc.JdbcValueConverters.convertDecimal(JdbcValueConverters.java:1016)
      at io.debezium.connector.oracle.OracleValueConverters.convertDecimal(OracleValueConverters.java:292)
      at io.debezium.connector.oracle.OracleValueConverters.convertNumeric(OracleValueConverters.java:297)
      at io.debezium.connector.oracle.OracleValueConverters.convertVariableScale(OracleValueConverters.java:390)
      at io.debezium.connector.oracle.OracleValueConverters.lambda$getNumericConverter$13(OracleValueConverters.java:203)
      at io.debezium.relational.TableSchemaBuilder.lambda$createKeyGenerator$3(TableSchemaBuilder.java:191)
      at io.debezium.relational.TableSchema.keyFromColumnData(TableSchema.java:130)
      at io.debezium.relational.RelationalChangeRecordEmitter.emitCreateRecord(RelationalChangeRecordEmitter.java:65)
      at io.debezium.relational.RelationalChangeRecordEmitter.emitChangeRecords(RelationalChangeRecordEmitter.java:45)
      at io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:193)
      ... 5 more
      2020-11-09 11:59:18,427 ERROR || WorkerSourceTask{id=LOGON-CONNECTOR-0} Task is being killed and will not recover until manually restarted [org.apache.kafka.connect.runtime.WorkerTask]
      

      Attachments

        Issue Links

          Activity

            People

              ccranfor@redhat.com Chris Cranford
              trustokoroego Trust Okoroego (Inactive)
              Votes:
              3 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: