[ https://issues.apache.org/jira/browse/KAFKA-10179?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17140158#comment-17140158 ]
Rohan Desai commented on KAFKA-10179: ------------------------------------- Deserialization may itself be a transformation. For example, suppose I have source data with 10 fields, but only care about 3 of them for my stream processing app. It seems that it would be reasonable to provide a deserializer that just extracts those 3 fields. I suppose you could express this as a projection after creating the table, but that does preclude optimizations that use selective deserialization. And it may be much more expensive to do the materialization (since you're potentially materializing lots of data unnecessarily). I think there should be some way to achieve each of the following: * optimized and the data in the store is exactly the same as the topic data . In this case (what's implemented today) the data can be restored by writing the source records into the store * optimized and the deserializer transforms the data somehow. In this case the data can be restored by deserializing/serializing each row from the source topic before writing it into the store. I don't think this is possible today. * not optimized (which would you have a transforming deserializer and faster recovery, at the cost of extra data in kafka). I don't think this is possible today without turning all optimizations off. > This is a known issue and tracked via: >https://issues.apache.org/jira/browse/KAFKA-8037 ack - thanks! > State Store Passes Wrong Changelog Topic to Serde for Optimized Source Tables > ----------------------------------------------------------------------------- > > Key: KAFKA-10179 > URL: https://issues.apache.org/jira/browse/KAFKA-10179 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 2.5.0 > Reporter: Bruno Cadonna > Assignee: Bruno Cadonna > Priority: Major > Fix For: 2.7.0 > > > {{MeteredKeyValueStore}} passes the name of the changelog topic of the state > store to the state store serdes. Currently, it always passes {{<application > ID>-<store name>-changelog}} as the changelog topic name. However, for > optimized source tables the changelog topic is the source topic. > Most serdes do not use the topic name passed to them. However, if the serdes > actually use the topic name for (de)serialization, e.g., when Kafka Streams > is used with Confluent's Schema Registry, a > {{org.apache.kafka.common.errors.SerializationException}} is thrown. -- This message was sent by Atlassian Jira (v8.3.4#803005)