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

Date and Time values without timezones are not persisted correctly based on database.time_zone

XMLWordPrintable

    • Icon: Bug Bug
    • Resolution: Done
    • Icon: Major Major
    • 2.3.0.Final
    • 2.2.0.Final
    • jdbc-connector
    • None

      What Debezium connector do you use and what version?

      Source: MySqlConnector, 2.2.0.Final

      -> Kafka / Avro Serialization

      Sink: JdbcSinkConnector, 2.2.0.Final

       

      What is the connector configuration?

      Source (removed credentials, broker setting; had no mysql timezone settings on source):
          "compression.type": "lz4",
          "connector.class": "io.debezium.connector.mysql.MySqlConnector",
          "schema.history.internal.skip.unparseable.ddl": "true",
          "errors.log.enable": "true",
          "errors.log.include.messages": "true",
          "include.schema.changes": "false",
          "key.converter": "io.confluent.connect.avro.AvroConverter",
          "key.converter.schema.registry.url": "....",
          "key.converter.include.schema.changes": "true",
          "key.converter.include.schema.events": "true",
          "key.converter.schemas.enable": "true",
          "snapshot.locking.mode": "none",
          "snapshot.mode": "schema_only",
          "inconsistent.schema.handling.mode": "warn",
          "snapshot.include.collection.list": "...",
          "table.include.list": "...",
          "tasks.max": "1",
          "value.converter.schema.registry.url": "....",
          "value.converter": "io.confluent.connect.avro.AvroConverter",
          "value.converter.include.schema.changes": "true",
          "value.converter.include.schema.events": "true",
          "value.converter.schemas.enable": "true"

      Sink:
         "connector.class": "io.debezium.connector.jdbc.JdbcSinkConnector",
         "tasks.max": "1",
         "topics.regex": "...",
         "connection.url": "....",
         "database.time_zone": "Europe/Vienna",
         "auto.create": "false",
         "auto.evolve": "false",
         "insert.mode": "upsert",
         "delete.enabled": "true",
         "primary.key.mode": "record_key",
         "topics.regex": "...",
         "transforms": "tablename",
         "transforms.tablename.type": "org.apache.kafka.connect.transforms.RegexRouter",
         "transforms.tablename.regex": "< my prefix and server string >
      .([^.]+)",
         "transforms.tablename.replacement": "$1",
         "key.converter": "io.confluent.connect.avro.AvroConverter",
         "key.converter.schema.registry.url": "...",
         "key.converter.schemas.enable": "true",
         "value.converter": "io.confluent.connect.avro.AvroConverter",
         "value.converter.schema.registry.url": "...",
         "value.converter.schemas.enable": "true"

      What is the captured database version and mode of depoyment?

      source & sink have same DB server with different schemas:

      • MySql 8 / innodb 8.0.25-15 - Percona Server (GPL), Release 15, Revision a558ec2
      • Timezone CEST

      Source and Sink connector plugins are running in the same Docker container. The base is the debezium/connect:2.2.0.Final image.

      What behaviour do you expect?

      data in source and sink tables have identical values in mysql timestamp and time fields.

      What behaviour do you see?

      source DB table:
       - dt_changed_export field of type timestamp = '2023-04-27 16:00:00'
       - t_time field of type time = '16:00:00'

      sink DB table:
       - dt_changed_export field of type timestamp = '2023-04-27 16:00:00'
       - t_time field of type time = '17:00:00'

      So the timestamp is copied OK, but the time value seem to have some TZ offset.

      In the corresponding Kafka event I see following message values (taken from Repanda UI):
      "dt_changed_export": "2023-04-27T14:00:00Z"
      "t_time": 57600000000

      57600000000 / (3600 * 1000 * 1000) = 16, so the mysql source connector seems to take the type time without UTC conversion, but the JDBC sink considers the TZ offset (without daylight saving offset)

      The corresponding Avro schema tells me:

      {
      "name":"dt_changed_export"
      "type":{
          "type":"string"
          "connect.version":1
          "connect.default":"1970-01-01T00:00:00Z"
          "connect.name":"io.debezium.time.ZonedTimestamp"
          }
      "default":"1970-01-01T00:00:00Z"
      }
      ...
      {
      "name":"t_time"
      "type":{
          "type":"long"
          "connect.version":1
          "connect.name":"io.debezium.time.MicroTime"
          }
      }

       
      Notice: Just as a local quick hack I replaced here 
      https://github.com/debezium/debezium-connector-jdbc/blob/main/src/main/java/io/debezium/connector/jdbc/type/debezium/MicroTimeType.java#L63
      the 
      query.setParameter(index, toZonedDateTime((long) value), StandardBasicTypes.ZONED_DATE_TIME_WITHOUT_TIMEZONE);
      with 
      query.setParameter(index, toZonedDateTime((long) value), StandardBasicTypes.ZONED_DATE_TIME_WITH_TIMEZONE);
      which seem to help in my case.  No idea if this can be the final fix.

      Do you see the same behaviour using the latest relesead Debezium version?

      I guess this is the latest released one.

      Do you have the connector logs, ideally from start till finish?

      (You might be asked later to provide DEBUG/TRACE level log)

      not yet.

      How to reproduce the issue using our tutorial deployment?

      did not try it.

        1. screenshot-1.png
          screenshot-1.png
          241 kB
        2. screenshot-2.png
          screenshot-2.png
          52 kB
        3. screenshot-3.png
          screenshot-3.png
          52 kB
        4. screenshot-4.png
          screenshot-4.png
          212 kB
        5. screenshot-5.png
          screenshot-5.png
          54 kB

            ccranfor@redhat.com Chris Cranford
            DetlefPumpenschnabel Detlef Pumpenschnabel (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

              Created:
              Updated:
              Resolved: