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

Postgres schema changes detection (not-null constraint)

    XMLWordPrintable

Details

    Description

      Hi.
      (Bad English ahead, thanks for understanding)
      I have encountered some interesting connector behavior that may not be a bug but feature, though i have to made a fix for myself.
      So - when i am changing not-null constraint on one of the columns in any table i can not detect that change in the message from the connector.

      Decoded message from the wa2json plugin looks like that:

      {
              "change": [
                      {
                              "kind": "update",
                              "schema": "public",
                              "table": "role",
                              "columnnames": ["id", "name", "isfalse"],
                              "columntypes": ["bigint", "character varying(255)", "boolean"],
                              "columnoptionals": [false, false, true],
                              "columnvalues": [13, "test", true],
                              "oldkeys": {
                                      "keynames": ["id", "name", "isfalse"],
                                      "keytypes": ["bigint", "character varying(255)", "boolean"],
                                      "keyvalues": [13, "test", false]
                              }
                      }
              ]
      }
      {
              "change": [
              ]
      }
      {
              "change": [
                      {
                              "kind": "update",
                              "schema": "public",
                              "table": "role",
                              "columnnames": ["id", "name", "isfalse"],
                              "columntypes": ["bigint", "character varying(255)", "boolean"],
                              "columnoptionals": [false, true, true],
                              "columnvalues": [13, null, true],
                              "oldkeys": {
                                      "keynames": ["id", "name", "isfalse"],
                                      "keytypes": ["bigint", "character varying(255)", "boolean"],
                                      "keyvalues": [13, "test", true]
                              }
                      }
              ]
      }
      
      • Where first message is a row data change, has "columnoptionals": [false, false, true]
      • Second message is schema change itself, second column has become not-null
      • Third message is again a row data change with "columnoptionals": [false, true, true]
        That is perfectly valid and i have planned to use that data to actually update schema in another DB.

      But the last message that was returned from the connector does not contains that schema changes. I can append serialized messages, but they are equal in the 'schema 'part.

      That happens because of the mechanism responsible for detection schema differences. RecordsStreamProducer:schemaChanged perform checks on type, length and scale of schema fields, but totally skips isOptional attribute. Result of this method is used later for schema refresh in the cache. And finally, in the RecordStreamProducer:generateUpdateRecord schema is retrieved from that unrefreshed cache and serialized into the outcoming connector message.

      I have found that suspicious since by default wal2json message has not so many fields to compare, which is "names", "types" and "optionals", where latter is totally skipped.

      Fix is quite simple - add

                      final boolean localOptional = column.isOptional();
                      final boolean incomingOptional = message.isOptional();
                      if (localOptional != incomingOptional) {
                          logger.info("detected new optional status for column '{}', old value was {} ({}), new value is {} ({}); refreshing table schema", columnName, localOptional, incomingOptional);
                          return true;
                      }
      

      somewhere into the RecordsStreamProducer:schemaChanged

      Attachments

        Activity

          People

            jpechane Jiri Pechanec
            zibranki Ilia Bogdanov (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: