-
Bug
-
Resolution: Done
-
Major
-
1.4.0.Alpha1
-
None
-
False
-
False
-
Undefined
-
-
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]
- is related to
-
DBZ-2967 Consolidate like code and refactor OracleChangeRecordValueConverter with OracleValueConverter
- Closed