[Debezium MySQL Connector] Avro compatibility issues upgrading from v1.4.1 to v1.5.X and newer

838 views
Skip to first unread message

Eric Weaver

unread,
Feb 2, 2022, 5:50:01 PM2/2/22
to debezium
Hi all,

We are currently trying to upgrade some Debezium mysql connectors from v1.4.1 to ideally v1.8 but are having trouble upgrading to anything past 1.4.2.

The release notes for 1.5.0.Alpha1 [1] mention breaking changes in the Avro schema for the schema change topic and have upgrade notes stating that the Avro schema compatibility must be set to FORWARD for new table schemas to be registered.

In our testing we're seeing new table schemas still failing to register in schema registry even after changing the compatibility mode to FORWARD. We were also able to reproduce this with the mysql Avro tutorials in debezium-examples [2] with the following steps

# Start connect and a debezium connector on 1.4
export DEBEZIUM_VERSION=1.4

docker-compose -f docker-compose-mysql-avro-worker.yaml up

curl -i -X POST -H "Accept:application/json" -H  "Content-Type:application/json" http://localhost:8083/connectors/ -d @register-mysql.json

# Stop the connector and connect worker for upgrade

curl -i -X DELETE -H "Accept:application/json" -H  "Content-Type:application/json" http://localhost:8083/connectors/inventory-connector

docker-compose -f docker-compose-mysql-avro-worker.yaml stop connect

# Change schema-registry to FORWARD compatibility (All schemas for convenience, but should only need dbserver1-value)

curl -XPUT localhost:8081/config -d '{"compatibility":"FORWARD"}' -H 'Content-Type: application/json'

# Upgrade to a newer debezium version

export DEBEZIUM_VERSION=1.5

docker-compose -f docker-compose-mysql-avro-worker.yaml up --no-deps connect

# Perform a DDL change

docker-compose -f docker-compose-mysql.yaml exec mysql bash -c 'mysql -u $MYSQL_USER -p$MYSQL_PASSWORD inventory -e "ALTER TABLE customers ADD COLUMN test VARCHAR(255) DEFAULT NULL;"'

# Start the connector again
curl -i -X POST -H "Accept:application/json" -H  "Content-Type:application/json" http://localhost:8083/connectors/ -d @register-mysql.json

At this point we are seeing the following exception in the connect logs:

Caused by: org.apache.kafka.common.config.ConfigException: Invalid value org.apache.kafka.common.errors.InvalidConfigurationException: Schema being registered is incompatible with an earlier schema for subject "dbserver1-value"; error code: 409 for configuration Failed to access Avro data from topic dbserver1 :
at io.confluent.connect.avro.AvroConverter.fromConnectData(AvroConverter.java:98)
at org.apache.kafka.connect.storage.Converter.fromConnectData(Converter.java:63)
at org.apache.kafka.connect.runtime.WorkerSourceTask.lambda$convertTransformedRecord$2(WorkerSourceTask.java:314)
at org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator.execAndRetry(RetryWithToleranceOperator.java:156)
at org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator.execAndHandleError(RetryWithToleranceOperator.java:190)
... 11 more


If I diff the v1.4 avro schema against the v1.5+ avro schema I see the expected "tableChanges" additions, but I'm also seeing a change in type from string to a union of string|null in two fields that seem to be causing this schema incompatibility

(left is v1.5, right is v1.4)
 {                                    {
      "name": "databaseName",           "name": "databaseName",
      "type": [                 |       "type": "string"
        "null",                 <
        "string"                <
      ],                        <
      "default": null           <
    },                                },
    {                                 {
      "name": "ddl",                    "name": "ddl",
      "type": [                 |       "type": "string"
        "null",                 <
        "string"                <
      ],                        <
      "default": null           <
    },                                },



Is there any guidance on how best we could proceed here? Are we missing a step in this upgrade path? One thought was to disable compatibility checks fully on this topic, allow the new schema to get registered and then attempt to turn on FORWARD compatibility again but we are unsure if this is a plausible or desired approach

Gunnar Morling

unread,
Feb 4, 2022, 12:35:36 PM2/4/22
to debezium
Hey Eric,

Interesting, I don't see a field "type" in the value schema of schema change events, neither in 1.4 nor in 1.5. Can you perhaps use JSON instead of Avro and share the emitted schema change events for both versions so we can compare the full schemas you see?

> One thought was to disable compatibility checks fully on this topic, allow the new schema to get registered and then attempt to turn on FORWARD compatibility again but we are unsure if this is a plausible or desired approach

I think that is a reasonable approach (you'd just have to make sure though to receive a schema change event using the new schema, before going back to the previous compat setting). I'd still like to understand though why this is happening. It may well be that we've messed up with that change back then, but I'm still not seeing the full picture.

Thanks,

--Gunnar
Message has been deleted
Message has been deleted
Message has been deleted

Gunnar Morling

unread,
Feb 11, 2022, 10:52:04 AM2/11/22
to debe...@googlegroups.com
Hey Eric,

Sorry for the delay, it seems there was a hick-up with Google Groups. So indeed you'll have to set NONE as compat level for that update. By making those existing fields optional and adding a new field at the same time, this schema change is neither forward nor backwards compatible. The described procedure of temporarily changing the compatibility settings for the migration should do the trick.

Best,

--Gunnar


Am Fr., 11. Feb. 2022 um 16:42 Uhr schrieb 'Eric Weaver' via debezium <debe...@googlegroups.com>:
Hey Gunnar,

It looks like my previously reply here got deleted. Sending again,


> Interesting, I don't see a field "type" in the value schema of schema change events, neither in 1.4 nor in 1.5.

Apologies I might have been unclear here. There isn't a field named "type". The existing fields "databaseName" and "ddl" appear to have their data types changed between 1.4 and 1.5. In 1.4 both of these fields were Avro STRING types and required values. In 1.5 these two fields, "databaseName" and "ddl" appear to be Avro UNION types, consisting of STRING|NULL types and defaulting NULL. This is where we are seeing unexpected Avro compatibility issues when using FORWARD compatibility as the upgrade notes state.


> Can you perhaps use JSON instead of Avro and share the emitted schema change events for both versions so we can compare the full schemas you see?

Yes, I was able to reproduce these schema differences between v1.4 and v1.5 using the JSON example setup as well.

The same steps as above were used to run the test using "docker-compose-mysql.yaml" docker-compose file instead of the "docker-compose-mysql-avro-worker.yaml" Avro worker config.

I run the following to get the JSON schemas and compare the schemas from the "customers" table from a 1.4 version to the 1.5 version

docker-compose -f docker-compose-mysql.yaml exec kafka /kafka/bin/kafka-console-consumer.sh     --bootstrap-server kafka:9092     --from-beginning     --property print.key=true     --topic dbserver1 | grep "customers"

I will attach the full schemas to the file as perhaps something in those were flagging the previous email.

The abbreviated changes can be seen here

# DEBEZIUM_VERSION=1.4
jq -f debezium-schema-v1.4.json 'select(.schema.name =="io.debezium.connector.mysql.SchemaChangeValue") | .. | select(.field? == "databaseName" or .field? == "ddl")'
{
  "type": "string",
  "optional": false,
  "field": "databaseName"
}
{
  "type": "string",
  "optional": false,
  "field": "ddl"
}


# DEBEZIUM_VERSION=1.5
jq -f debezium-schema-v1.5.json 'select(.schema.name =="io.debezium.connector.mysql.SchemaChangeValue") | .. | select(.field? == "databaseName" or .field? == "ddl")'
{
  "type": "string",
  "optional": true,
  "field": "databaseName"
}
{
  "type": "string",
  "optional": true,
  "field": "ddl"
}

This looks to me like it affects both the Avro and JSON schemas. Please let me know if there is more information I can help provide.

Thank you,
Eric

--
You received this message because you are subscribed to the Google Groups "debezium" group.
To unsubscribe from this group and stop receiving emails from it, send an email to debezium+u...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/04edfd29-8d2b-4ed6-92cb-84d1358f6368n%40googlegroups.com.

Eric Weaver

unread,
Feb 14, 2022, 11:03:26 AM2/14/22
to debezium
Hey Gunnar,

Thank you for verifying my understanding of the problem and path forward here. I know the v1.5 release is quite a ways back but would there be interest in updating the older release notes here [0] (and elsewhere if needed) to reflect the need to turn schema compatibility to NONE temporarily for upgrading when using AVRO so other's don't trip on this? Happy to file a Jira issue and submit a pull request if so.

[0] https://github.com/debezium/debezium.github.io/blob/develop/releases/1.5/release-notes.asciidoc#breaking-changes-8

Thanks again,
Eric

Gunnar Morling

unread,
Feb 15, 2022, 3:14:49 AM2/15/22
to debezium
Hey Eric,

Yes, such contribution for updating the docs would definitely be welcomed. No Jira issue needed, just use the "[docs]" prefix for your commit message, e.g. "[docs] Updating migration notes to ...".

Thanks a lot,

--Gunnar
Reply all
Reply to author
Forward
0 new messages