[
https://issues.apache.org/jira/browse/KAFKA-20179?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-20179:
------------------------------------
Component/s: streams
> Avoiding headers deserialization while changelogging
> ----------------------------------------------------
>
> Key: KAFKA-20179
> URL: https://issues.apache.org/jira/browse/KAFKA-20179
> Project: Kafka
> Issue Type: Sub-task
> Components: streams
> Reporter: Alieh Saeedi
> Priority: Major
>
> When writing to the changelog topic, we currently need to deserialize the
> record headers, which is not ideal from a performance perspective. We should
> look for a way to avoid this overhead.
> One option, as suggested by [~mjsax] , would be to reuse the header bytes
> from the Kafka message format directly, without deserialization. However,
> this would couple Kafka Streams tightly to the internal serialization format
> of headers in the Kafka record. Although we already use the same format
> today, we are not yet _formally_ coupled to it. If the Kafka message format
> for headers ever changes, this approach would directly impact Kafka Streams
> and raise compatibility concerns (ref:
> [https://github.com/apache/kafka/pull/21345/changes#r2761685550)|https://github.com/apache/kafka/pull/21345/changes#r2761685550].
> Another complication is that the processor context can still mutate
> headers—for example, by adding new entries—before the record is sent to the
> producer (see {{{}[ProcessorContextImpl#log():
> https://github.com/apache/kafka/blob/6d9ba767c544d600acbcc1eec8dd38d94c739b01/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java#L136{}}}).
> This leaves us with two options:
> # Deserialize the headers anyway, which means we lose the optimization; or
> # Manually merge: serialize any newly added header entries (e.g., the vector
> clock), append them to the existing serialized headers, and then send the
> combined header bytes to the producer.
> Neither option is entirely straightforward, so we need to weigh the
> performance gains of avoiding deserialization against the tighter coupling
> and added complexity this would introduce.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)