-
Bug
-
Resolution: Unresolved
-
Major
-
None
-
None
-
False
-
-
False
-
Important
Bug report
The Debezium MariaDB source connector encounters a schema inconsistency error when a table includes a generated column that is part of a unique key constraint. In such cases, MariaDB automatically creates a hidden index column to support the constraint, which is not reflected in the table schema retrieved by Debezium. However, during CDC processing, Debezium receives before and after row data that includes this hidden/generated column, causing a mismatch between the internal schema and the binlog row data.
What Debezium connector do you use and what version?
- Connector: io.debezium.connector.mariadb.MariaDbConnector
- Version: 3.0.7.Final
What is the connector configuration?
{ "connector.class": "io.debezium.connector.mariadb.MariaDbConnector", "snapshot.locking.mode": "none", "errors.log.include.messages": "false", "transforms": "unwrap", "transforms.unwrap.drop.tombstones": "true", "transforms.unwrap.type": "io.debezium.transforms.ExtractNewRecordState", "errors.log.enable": "true", "key.converter": "io.confluent.connect.avro.AvroConverter", "transforms.unwrap.field.name.adjustment.mode": "avro", "errors.tolerance": "none", "max.batch.size": "2048", "snapshot.mode": "no_data", "connect.timeout.ms": "10000", "transforms.unwrap.delete.handling.mode": "rewrite", "max.queue.size": "8192", "incremental.snapshot.chunk.size": "32768", "transforms.unwrap.schema.name.adjustment.mode": "avro", "schema.history.internal.store.only.captured.databases.ddl": "true", "schema.history.internal.store.only.captured.tables.ddl": "true", "binary.handling.mode": "bytes", "value.converter": "io.confluent.connect.avro.AvroConverter", "value.converter.ignore.default.for.nullables": "true", "read.only": "false", "converters": "JdbcSinkDataTypesConverter", "JdbcSinkDataTypesConverter.type": "io.debezium.connector.binlog.converters.JdbcSinkDataTypesConverter", "JdbcSinkDataTypesConverter.selector.real": "", "JdbcSinkDataTypesConverter.selector.boolean": "", "transforms.unwrap.add.headers": "op", "database.port": "3306", "database.ssl.mode": "disable", "database.connectionTimeZone": "SERVER", "database.hostname": "mariadb", "transforms.unwrap.add.fields": "source.ts_ms,ts_ms", "database.useGtid": "none", "table.include.list": "mydatabase.tbl_settings", "key.converter.schema.registry.url": "...", … }
What is the captured database version and mode of deployment?
- Database: MariaDB (10.6.19).
- Deployment: cloud & docker.
What behavior do you expect?
Debezium should either ignore the hidden generated columns or properly account for them in its internal schema representation to prevent schema mismatch errors.
What behavior do you see?
Debezium MariaDB connector fails with a schema mismatch when a generated column is used in a unique constraint—MariaDB creates a hidden column (e.g., DB_ROW_HASH_*) for the supporting index, which is excluded from the snapshot but appears in binlog row data, causing inconsistency.
Do you see the same behaviour using the latest released Debezium version?
Yes
Do you have the connector logs, ideally from start till finish?
Task threw an uncaught and unrecoverable exception. Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask) org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped. at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleEvent(BinlogStreamingChangeEventSource.java:591) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.lambda$execute$17(BinlogStreamingChangeEventSource.java:209) at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1281) at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1103) at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:657) at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:959) at java.base/java.lang.Thread.run(Thread.java:840) Caused by: io.debezium.DebeziumException: Error processing binlog event ... 7 more Caused by: io.debezium.DebeziumException: Error processing row in tbl_settings, internal schema size 17, but row size 18 , restart connector with schema recovery mode. at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.validateChangeEventWithTable(BinlogStreamingChangeEventSource.java:1065) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.lambda$handleInsert$28(BinlogStreamingChangeEventSource.java:837) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleChange(BinlogStreamingChangeEventSource.java:1101) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleInsert(BinlogStreamingChangeEventSource.java:832) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.lambda$execute$7(BinlogStreamingChangeEventSource.java:183) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleEvent(BinlogStreamingChangeEventSource.java:571) ... 6 more
How to reproduce the issue using our tutorial deployment?
- Create a MariaDB table with a generated column used in a unique key constraint
- Insert or update data in the table
- Observe Debezium connector logs and failure behavior
Implementation ideas (optional)
We introduced a configurable workaround to optionally ignore schema inconsistency errors caused by hidden columns like DB_ROW_HASH_*. A new flag ignore.schema.row.inconsistency was added, along with support for a list of specific tables to ignore (ignore.schema.row.inconsistency.tables). When enabled, the connector skips schema validation and logs a warning if the number of columns in binlog data doesn't match the internal schema, allowing CDC to continue without failure.
- is duplicated by
-
DBZ-9444 MariaDB connector get schema mismatch for system-versioned table
-
- Closed
-