Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Kafka already has serializers and deserializers for primitive numeric values, such as IntegerSerializer and IntegerDeserializer. Connect doesn't yet have Converter implementations for these primitives, yet it may be useful for some connectors to use numbers as keys or values within records. In these cases, primitive converters are more useful and desirable than having to use the String, JSON, or Avro converters.

Note that primitive integral numbers makes sense for keys, whereas floating point numbers much much less sense. Despite that, this proposal suggests adding Converters for all of Kafka's existing primitive serdes, keeping symmetry between the available options.

Public Interfaces

Add five new implementations of the existing org.apache.kafka.connect.storage.Converter interface, which use the corresponding (existing) Kafka serializer and deserializers that already support nulls. All of the Converter implementations will allow serializing and deserializing null values, and will always use the specified Connect optional schema when deserializing values to allow for nulls. These classes will also implement the existing org.apache.kafka.connect.storage.HeaderConverter interface, which like the existing StringConverter simply delegates to the normal Converter methods. The topic and header names supplied to the converters do not affect serialization or deserialization.

Class nameDeserialized SchemaUses existing SerializerUses existing Deserializer

org.apache.kafka.connect.storage.ShortConverter

Schema.OPTIONAL_INT16
org.apache.kafka.common.serialization.ShortSerializer
org.apache.kafka.common.serialization.ShortDeserializer

org.apache.kafka.connect.storage.IntegerConverter

Schema.OPTIONAL_INT32
org.apache.kafka.common.serialization.IntegerSerializer
org.apache.kafka.common.serialization.IntegerDeserializer

org.apache.kafka.connect.storage.LongConverter

Schema.OPTIONAL_INT64
org.apache.kafka.common.serialization.LongSerializer
org.apache.kafka.common.serialization.LongDeserializer

org.apache.kafka.connect.storage.FloatConverter

Schema.OPTIONAL_FLOAT32
org.apache.kafka.common.serialization.FloatSerializer
org.apache.kafka.common.serialization.FloatDeserializer

org.apache.kafka.connect.storage.DoubleConverter

Schema.OPTIONAL_FLOAT64
org.apache.kafka.common.serialization.DoubleSerializer
org.apache.kafka.common.serialization.DoubleDeserializer

...

A second alternative that was considered was to use one Converter that could dynamically determine the serialized and deserialized form for each key or value. This was rejected since no corresponding Kafka serdes implementation would work the same way.

Third, it was considered to only provide ShortConverter, IntegerConverter, and LongConverter since integral numbers make sense as message keys. In fact, one might argue that only longs make sense as keys, since the shorts and integers have too small a range to be useful as keys. However, the proposal still includes all five Converters (a) to maintain symmetry with the existing Kafka serdes, and (b) to allow them to be used as HeaderConverters. They are very simple classes, so the maintenance cost is minor. A bigger question is whether the additional converters lead users to use them inappropriately, but here the lack of symmetry with existing Kafka serdes might be a bigger concern.