[
https://issues.apache.org/jira/browse/KAFKA-20179?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Alieh Saeedi updated KAFKA-20179:
---------------------------------
Description:
When writing to the changelog topic, we currently need to deserialize the
headers, which is not ideal. We should find a way to avoid this for efficiency
reasons.
Suggested solution by [~mjsax]: `We would couple ourselves strictly to the
serialization format of headers inside the message format thought. While we
already use the same format atm we are not tightly coupled atm. So we should
consider the implications if we want to do go down this route. For the case
that this part of the Kafka message format changes, it would impact KS, and
this raised compatibility questions` (ref:
[https://github.com/apache/kafka/pull/21345/changes#r2761685550)|https://github.com/apache/kafka/pull/21345/changes#r2761685550]
Another challenge is that the processor context might still modify the
headers—by adding new entries—before the record is sent to the producer:
[https://github.com/apache/kafka/blob/6d9ba767c544d600acbcc1eec8dd38d94c739b01/stre[…]che/kafka/streams/processor/internals/ProcessorContextImpl.java|https://github.com/apache/kafka/blob/6d9ba767c544d600acbcc1eec8dd38d94c739b01/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java#L136]
So either 1) we have to deserialize the headers anyway (no optimization) or 2)
serialize the added things (vector clock in this case) and add them to our
serialized headers and then send them.
was:
When writing to the changelog topic, we currently need to deserialize the
headers, which is not ideal. We should find a way to avoid this for efficiency
reasons.
Suggested solution by [~mjsax]: `We would couple ourselves strictly to the
serialization format of headers inside the message format thought. While we
already use the same format atm we are not tightly coupled atm. So we should
consider the implications if we want to do go down this route. For the case
that this part of the Kafka message format changes, it would impact KS, and
this raised compatibility questions.`
ref: https://github.com/apache/kafka/pull/21345/changes#r2761685550
> Avoiding headers deserialization while changelogging
> ----------------------------------------------------
>
> Key: KAFKA-20179
> URL: https://issues.apache.org/jira/browse/KAFKA-20179
> Project: Kafka
> Issue Type: Sub-task
> Reporter: Alieh Saeedi
> Priority: Major
>
> When writing to the changelog topic, we currently need to deserialize the
> headers, which is not ideal. We should find a way to avoid this for
> efficiency reasons.
> Suggested solution by [~mjsax]: `We would couple ourselves strictly to the
> serialization format of headers inside the message format thought. While we
> already use the same format atm we are not tightly coupled atm. So we should
> consider the implications if we want to do go down this route. For the case
> that this part of the Kafka message format changes, it would impact KS, and
> this raised compatibility questions` (ref:
> [https://github.com/apache/kafka/pull/21345/changes#r2761685550)|https://github.com/apache/kafka/pull/21345/changes#r2761685550]
> Another challenge is that the processor context might still modify the
> headers—by adding new entries—before the record is sent to the producer:
> [https://github.com/apache/kafka/blob/6d9ba767c544d600acbcc1eec8dd38d94c739b01/stre[…]che/kafka/streams/processor/internals/ProcessorContextImpl.java|https://github.com/apache/kafka/blob/6d9ba767c544d600acbcc1eec8dd38d94c739b01/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java#L136]
> So either 1) we have to deserialize the headers anyway (no optimization) or
> 2) serialize the added things (vector clock in this case) and add them to our
> serialized headers and then send them.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)