Status
Current state: Under Discussion
Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]
JIRA: here [Change the link from KAFKA-1 to your own ticket]
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Most JSON data that utilizes precise decimal data represents it as a decimal string. Connect, on the other hand, only supports a binary HEX string encoding (see example below). This KIP intends to support all three of the below data types:
{ "asHex": "D3J5", "asString": "10.12345" "asNumber": 10.2345 }
Public Interfaces
Two new configurations will be added to the JsonConverterConfig:
- json.decimal.serialization.format AND
- json.decimal.deserialization.format
Both of these will be string values that are managed by a new enumeration:
public enum SerializationFormat { BINARY, TEXT, NUMERIC; public static SerializationFormat forName(String name) { return SerializationFormat.valueOf(name.toUpperCase(Locale.ROOT)); } }
As of this change, only BINARY, TEXT and NUMERIC values will be supported. The defaults for both of these values will be BINARY to maintain backwards compatibility.
Proposed Changes
JsonConverter will be configurable with the new values. If the values are present, then it will attempt to serialize and deserialize the input values based on the configuration values listed above respectively.
Compatibility, Deprecation, and Migration Plan
This change is backwards compatible, and no functionality will be deprecated. Users must be careful when enabling the new serialization functionality to ensure that all downstream data consumers can read data serialized in the new format.
Rejected Alternatives
- Encoding the serialization in the schema for Decimal LogicalType. This is good because it means that the deserializer will be able to decode based on the schema and one converter can handle different topics encoded differently as long as the schema is in line. The problem is that this is specific to only JSON and changing the LogicalType is not the right place.