[ https://issues.apache.org/jira/browse/KAFKA-5632?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16266322#comment-16266322 ]
Matthias J. Sax commented on KAFKA-5632: ---------------------------------------- No update. You are referring to KAFKA-5311 with regard to {{ExtendedDeserializer}}. As you can see from the JIRA description, handling headers in not straight forward... [~guozhang] [~damianguy] [~bbejeck] With regard to KIP-159 discussion, I think we face a similar issue here. Maybe, we could avoid this, by exposing the headers only in source operators and allow users to specify headers manually in sink operators. If anybody wants to carry header information, they need to picky back them in the value in the source operator. Just a thought. Difference to KIP-159 is, that in KIP-159 we only have read path -- here, we need a write path to set headers for output topics, too. > Message headers not supported by Kafka Streams > ---------------------------------------------- > > Key: KAFKA-5632 > URL: https://issues.apache.org/jira/browse/KAFKA-5632 > Project: Kafka > Issue Type: Bug > Components: consumer > Affects Versions: 0.11.0.0 > Reporter: CJ Woolard > Priority: Minor > Labels: needs-kip > > The new message headers functionality introduced in Kafka 0.11.0.0 > (https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers) > does not appear to be respected by Kafka Streams, specifically message > headers set on input topics to a Kafka Streams topology do not get propagated > to the corresponding output topics of the topology. > It appears that it's at least partially due to the > SourceNodeRecordDeserializer not properly respecting message headers here: > https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializer.java#L60 > where it isn't using the new ConsumerRecord constructor which supports > headers: > https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java#L122 > For additional background here is the line before which we noticed that we > still have the message headers, and after which we no longer have them: > https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java#L93 > In terms of a potential solution there are a few different scenarios to > consider: > 1. A stream processor with one input and one output, i.e. 1-to-1, (A > map/transformation for example). This is the simplest case, and one proposal > would be to directly propagate any message headers from input to output. > 2. A stream processor with one input and many outputs, i.e. 1-to-many, (A > flatmap step for example). > 3. A stream processor with multiple inputs per output, i.e. many-to-1, (A > join step for example). > One proposal for supporting all possible scenarios would be to expose > overloads in the Kafka Streams DSL methods to allow the user the ability to > specify logic for handling of message headers. > For additional background the use case is similar to a distributed tracing > use case, where the following previous work may be useful for aiding in > design discussions: > Dapper > (https://static.googleusercontent.com/media/research.google.com/en//pubs/archive/36356.pdf) > > or > Zipkin (https://github.com/openzipkin/zipkin) -- This message was sent by Atlassian JIRA (v6.4.14#64029)