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

MySQL connector fails on a zero date

    XMLWordPrintable

Details

    • Bug
    • Resolution: Done
    • Major
    • 1.4.0.Alpha2
    • None
    • mysql-connector
    • None
    • False
    • False
    • Undefined
    • Hide
      1. Grab the example setup for MySQL from the tutorial.
      2. Set sql-mode= in MySQL configuration.
      3. Add "column.truncate.to.65536.chars": "^\\w+.
        w+.w+"
        to the connector configuration.
      4. Log in to the MySQL database.
      5. Create a new DATE NOT NULL column in a table with existing data:
        alter table customers add column date_c date not null
      6. Update an existing record:
        update customers set first_name='N' where id=1001

      See the connector fail:

       connect_1 | Caused by: org.apache.kafka.connect.errors.DataException: Invalid value: null used for required field: "date_c", schema type: INT32
       connect_1 | at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:220)
       connect_1 | at org.apache.kafka.connect.data.Struct.validate(Struct.java:233)
       connect_1 | at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:253)
       connect_1 | at org.apache.kafka.connect.data.Struct.put(Struct.java:216)
       connect_1 | at org.apache.kafka.connect.data.Struct.put(Struct.java:203)
       connect_1 | at io.debezium.data.Envelope.update(Envelope.java:313)
       connect_1 | at io.debezium.connector.mysql.RecordMakers$1.update(RecordMakers.java:314)
       connect_1 | at io.debezium.connector.mysql.RecordMakers$RecordsForTable.update(RecordMakers.java:512)
       connect_1 | at io.debezium.connector.mysql.BinlogReader.handleUpdate(BinlogReader.java:992)
       connect_1 | at io.debezium.connector.mysql.BinlogReader.handleEvent(BinlogReader.java:587)
       connect_1 | ... 5 more
        

      What happens under the hood is:

      1. When a new DATE NOT NULL column is added, MySQL will populate it with zero dates on the existing rows:
        mysql> select * from customers;
        +------+------------+-----------+-----------------------+------------+
        | id   | first_name | last_name | email                 | date_c     |
        +------+------------+-----------+-----------------------+------------+
        | 1001 | M          | Thomas    | sally.thomas@acme.com | 0000-00-00 |
        | 1002 | George     | Bailey    | gbailey@foobar.com    | 0000-00-00 |
        | 1003 | Edward     | Walker    | ed@walker.com         | 0000-00-00 |
        | 1004 | Anne       | Kretchmar | annek@noanswer.org    | 0000-00-00 |
        +------+------------+-----------+-----------------------+------------+
        4 rows in set (0.01 sec)
        
      1. When reading an empty date from the binlog, the connector will represent it as NULL and emit a warning:
        if (year == 0 || month == 0 || day == 0) {
            LOGGER.warn("Invalid value '{}' stored in column '{}' of table '{}' converted to empty value", dateString, column.name(), table.id());
            return null;
        }
        
      1. NULL doesn't fit the schema where this field is required.
      Show
      Grab the example setup for MySQL from the tutorial. Set  sql-mode= in MySQL configuration. Add "column.truncate.to.65536.chars": "^\\w+. w+.w+" to the connector configuration. Log in to the MySQL database. Create a new DATE NOT NULL column in a table with existing data: alter table customers add column date_c date not null Update an existing record: update customers set first_name='N' where id=1001 See the connector fail: connect_1 | Caused by: org.apache.kafka.connect.errors.DataException: Invalid value: null used for required field: "date_c", schema type: INT32 connect_1 | at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:220) connect_1 | at org.apache.kafka.connect.data.Struct.validate(Struct.java:233) connect_1 | at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:253) connect_1 | at org.apache.kafka.connect.data.Struct.put(Struct.java:216) connect_1 | at org.apache.kafka.connect.data.Struct.put(Struct.java:203) connect_1 | at io.debezium.data.Envelope.update(Envelope.java:313) connect_1 | at io.debezium.connector.mysql.RecordMakers$1.update(RecordMakers.java:314) connect_1 | at io.debezium.connector.mysql.RecordMakers$RecordsForTable.update(RecordMakers.java:512) connect_1 | at io.debezium.connector.mysql.BinlogReader.handleUpdate(BinlogReader.java:992) connect_1 | at io.debezium.connector.mysql.BinlogReader.handleEvent(BinlogReader.java:587) connect_1 | ... 5 more   What happens under the hood is: When a new DATE NOT NULL column is added, MySQL will populate it with zero dates on the existing rows: mysql> select * from customers; +------+------------+-----------+-----------------------+------------+ | id | first_name | last_name | email | date_c | +------+------------+-----------+-----------------------+------------+ | 1001 | M | Thomas | sally.thomas@acme.com | 0000-00-00 | | 1002 | George | Bailey | gbailey@foobar.com | 0000-00-00 | | 1003 | Edward | Walker | ed@walker.com | 0000-00-00 | | 1004 | Anne | Kretchmar | annek@noanswer.org | 0000-00-00 | +------+------------+-----------+-----------------------+------------+ 4 rows in set (0.01 sec) When reading an empty date from the binlog, the connector will represent it as NULL and emit a warning: if (year == 0 || month == 0 || day == 0) { LOGGER.warn( "Invalid value '{}' stored in column '{}' of table '{}' converted to empty value" , dateString, column.name(), table.id()); return null ; } NULL doesn't fit the schema where this field is required.

    Description

      If the source MySQL database has the strict mode disabled for date fields, the connector will fail when an empty date is encountered.

      Attachments

        Activity

          People

            Unassigned Unassigned
            sergeimorozov Sergei Morozov
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: