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

Define practical way to deal with temporal values

    XMLWordPrintable

Details

    • Bug
    • Resolution: Done
    • Major
    • 0.3
    • 0.3
    • mysql-connector
    • None

    Description

      Kafka Connect's logical temporal types in org.apache.kafka.connect.data include Time, Date, and Timestamp, and these logical types convert to/from java.util.Date and store values in events as either milliseconds (INT64) or epoch days (INT32). This technically works because none of these have any timezone information, but using java.util.Date is very difficult and error prone, especially when it involves determining the java.util.Date using year, month, day, hour, etc. values as that requires using a Calendar that if not properly used defaults to using the local time zone in conversions. The MySQL connector, for example, ends up using 4 possible timezone conversions between the database and consumer.

      Kafka Connect's built-in logical types are also problematic, since they require serialization and deserialization from/to java.util.Date, meaning that even if we calculate the correct primitive representation, to use these logical types we still need to construct a java.util.Date. We can use this internally, but if consumers do use the Kafka Connect converter libraries, they will get java.util.Date objects in their events. By defining our own semantic types, Debezium can completely avoid the conflagration of the serialization/deserialization logic with the semantic representation.

      Kafka Connect's built-in logical types are also restricted to millisecond precision, and have no support for times or timestamps that include timezone information. Meanwhile, Debezium can define separate semantic temporal types for representing millisecond, microsecond, or nanosecond precisions, and we can define semantic types for times and timestamps that include timezone information.

      Debezium can also use Java 8's javax.time library to make conversion between MySQL raw binlog event values and the literals that go into Debezium-produced events, and completely eliminate the use of time zone conversions (except in MySQL TIMESTAMP columns that are stored in UTC). It should also provide a way to keep the precision of the database columns, which for MySQL 5.6.4 and up can use up to microsecond precision (where the fractional second precision is set to 6). This will involve defining additional semantic Kafka Connect schemas, customizing how the MySQL binlog client library converts raw binlog event data, and rewriting the temporal conversion logic.

      Kafka Connect's schema system makes it possible for us to define our own semantic types yet still represent data with primitives. To do this, a field's schema is defined with a type (e.g., INT32, INT64, STRING, etc.) and a name (e.g., org.apache.kafka.connect.data.Time or io.debezium.time.Time). Each converter is free to use this information, and the JSON converter and Avro converters both do. Specifically, the Avro Converter uses the type to define how the values are serialized and includes the schema name as the connect.name property in the Avro schema's field definition (which Avro doesn't use, though it is available for consumers that use the schemas).

      The MySQL connector's configuration will allow specifying whether to use the Kafka Connect built-in logical types or Debezium's semantic types that adapt based upon the column's precision.

      Attachments

        Issue Links

          Activity

            People

              rhauch Randall Hauch (Inactive)
              rhauch Randall Hauch (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: