Status
Current state: "Under Discussion"
Discussion thread: here
JIRA: here
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
The internal.key.converter and internal.value.converter were original exposed as configs because
- they are actually pluggable
- providing a default would require relying on the JsonConverter always being available, which until we had classloader isolation it was possible might be removed for compatibility reasons.
However, this has ultimately just caused a lot more trouble and confusion than it is worth.
Public Interfaces
WorkerConfig.java public static final String INTERNAL_KEY_CONVERTER_CLASS_CONFIG = "internal.key.converter"; public static final String INTERNAL_KEY_CONVERTER_CLASS_DOC = "Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka." + " This controls the format of the keys in messages written to or read from Kafka, and since this is" + " independent of connectors it allows any connector to work with any serialization format." + " Examples of common formats include JSON and Avro." + " This setting controls the format used for internal bookkeeping data used by the framework, such as" + " configs and offsets, so users can typically use any functioning Converter implementation."; public static final String INTERNAL_VALUE_CONVERTER_CLASS_CONFIG = "internal.value.converter"; public static final String INTERNAL_VALUE_CONVERTER_CLASS_DOC = "Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka." + " This controls the format of the values in messages written to or read from Kafka, and since this is" + " independent of connectors it allows any connector to work with any serialization format." + " Examples of common formats include JSON and Avro." + " This setting controls the format used for internal bookkeeping data used by the framework, such as" + " configs and offsets, so users can typically use any functioning Converter implementation."; connect-standalone.properties internal.key.converter=org.apache.kafka.connect.json.JsonConverter internal.value.converter=org.apache.kafka.connect.json.JsonConverter connect-distributed.properties internal.key.converter=org.apache.kafka.connect.json.JsonConverter internal.value.converter=org.apache.kafka.connect.json.JsonConverter
Proposed Changes
We should deprecate the above configs, give them a default of JsonConverter (which is also a kind of nice since it results in human-readable data in the internal topics), and then ultimately remove them in the next major version.
Compatibility, Deprecation, and Migration Plan
This is a backward compatible change and the configs would be supported until next major release.
Rejected Alternatives
None