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

Backwards incompatible schema changes in `sources` field

XMLWordPrintable

    • Icon: Bug Bug
    • Resolution: Won't Do
    • Icon: Blocker Blocker
    • None
    • 0.4
    • mysql-connector
    • None

      A backwards incompatible schema change was made in the 0.4.0 release. Prior to 0.4.0, the source had an Avro schema like so:

          {
            "name": "source",
            "type": {
              "type": "record",
              "name": "Source",
              "namespace": "io.debezium.connector.mysql",
              "fields": [
                {
                  "name": "name",
                  "type": "string"
                },
                {
                  "name": "server_id",
                  "type": "long"
                },
                {
                  "name": "ts_sec",
                  "type": "long"
                },
                {
                  "name": "gtid",
                  "type": [
                    "null",
                    "string"
                  ]
                },
                {
                  "name": "file",
                  "type": "string"
                },
                {
                  "name": "pos",
                  "type": "long"
                },
                {
                  "name": "row",
                  "type": "int"
                },
                {
                  "name": "snapshot",
                  "type": [
                    "null",
                    "boolean"
                  ]
                }
              ],
              "connect.name": "io.debezium.connector.mysql.Source"
            }
          },
      

      After 0.4.0, three new fields were added to the end:

          {
            "name": "source",
            "type": {
              "type": "record",
              "name": "Source",
              "namespace": "io.debezium.connector.mysql",
              "fields": [
                {
                  "name": "name",
                  "type": "string"
                },
                {
                  "name": "server_id",
                  "type": "long"
                },
                {
                  "name": "ts_sec",
                  "type": "long"
                },
                {
                  "name": "gtid",
                  "type": [
                    "null",
                    "string"
                  ]
                },
                {
                  "name": "file",
                  "type": "string"
                },
                {
                  "name": "pos",
                  "type": "long"
                },
                {
                  "name": "row",
                  "type": "int"
                },
                {
                  "name": "snapshot",
                  "type": [
                    "null",
                    "boolean"
                  ]
                },
                {
                  "name": "thread",
                  "type": [
                    "null",
                    "long"
                  ]
                },
                {
                  "name": "db",
                  "type": [
                    "null",
                    "string"
                  ]
                },
                {
                  "name": "table",
                  "type": [
                    "null",
                    "string"
                  ]
                }
              ],
              "connect.name": "io.debezium.connector.mysql.Source"
            }
          },
      

      Note the addition of thread, db, and table. This is problematic. The schemas for these three fields have:

                {
                  "name": "foo",
                  "type": [
                    "null",
                    "string"
                  ]
                }
      

      Note the absence of a default. This is not a backwards compatible change as defined by Confluent's schema registry. A backwards compatible change must allow for data produced by the old schema to be consumed by the new schema. The absence of a default in this case means that consumers using the new schema will expect values for the db/thread/table fields--even if they're null, they must be set. Old messages will have this field completely missing, which will lead to failures.

      Our schema registry is rejecting this change:

      [2017-02-10 00:49:04,007] INFO Source task WorkerSourceTask{id=debezium-connector-poc-0} finished initialization and start (org.apache.kafka.connect.runtime.WorkerSourceTask:138)
      [2017-02-10 00:49:06,087] ERROR Task debezium-connector-poc-0 threw an uncaught and unrecoverable exception (org.apache.kafka.connect.runtime.WorkerTask:142)
      org.apache.kafka.connect.errors.DataException: Failed to serialize Avro data: 
              at io.confluent.connect.avro.AvroConverter.fromConnectData(AvroConverter.java:92)
              at org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:183)
              at org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:160)
              at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:140)
              at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:175)
              at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
              at java.util.concurrent.FutureTask.run(FutureTask.java:266)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
              at java.lang.Thread.run(Thread.java:745)
      Caused by: org.apache.kafka.common.errors.SerializationException: Error registering Avro schema: {"type":"record","name":"Envelope","namespace":"db.debezium.apps","fields":[{"name":"before","type":["null",{"type":"record","name":"Value","fields":[{"name":"id","type":"long"},{"name":"name","type":["null","string"]},{"name":"user_id","type":["null","long"]},{"name":"group_id","type":["null","long"]},{"name":"ts_create_time","type":{"type":"string","connect.version":1,"connect.name":"io.debezium.time.ZonedTimestamp"}},{"name":"ts_modify_time","type":{"type":"string","connect.version":1,"connect.name":"io.debezium.time.ZonedTimestamp"}},{"name":"version","type":["null","int"]},{"name":"settings","type":["null",{"type":"string","connect.version":1,"connect.name":"io.debezium.data.Json"}]}],"connect.name":"apps.Value"}]},{"name":"after","type":["null","Value"]},{"name":"source","type":{"type":"record","name":"Source","namespace":"io.debezium.connector.mysql","fields":[{"name":"name","type":"string"},{"name":"server_id","type":"long"},{"name":"ts_sec","type":"long"},{"name":"gtid","type":["null","string"]},{"name":"file","type":"string"},{"name":"pos","type":"long"},{"name":"row","type":"int"},{"name":"snapshot","type":["null","boolean"]},{"name":"thread","type":["null","long"]},{"name":"db","type":["null","string"]},{"name":"table","type":["null","string"]}],"connect.name":"io.debezium.connector.mysql.Source"}},{"name":"op","type":"string"},{"name":"ts_ms","type":["null","long"]}],"connect.version":1,"connect.name":"apps.Envelope"}
      Caused by: io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException: Schema being registered is incompatible with the latest schema; error code: 409; error code: 409
      

              rhauch Randall Hauch (Inactive)
              criccomini Chris Riccomini (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

                Created:
                Updated:
                Resolved: